diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS new file mode 100644 index 0000000000000..40b578fe59555 --- /dev/null +++ b/.github/CODEOWNERS @@ -0,0 +1 @@ +* @streamnative/compute diff --git a/.github/README.md b/.github/README.md new file mode 100644 index 0000000000000..cb97d3c8652f4 --- /dev/null +++ b/.github/README.md @@ -0,0 +1,11 @@ +# StreamNative's fork of Apache Flink + +This repository is maintained by StreamNative to host a StreamNative fork of the Flink Connector for Pulsar. Only the Flink Connector for Pulsar (including the Flink SQL Connector for Pulsar and the Flink DataStream Connector for Pulsar) diverges with the official Apache Flink repository. + +The following is some useful documentation and resources for the DataStream and SQL Connectors + +- [Flink SQL Connector for Pulsar documentation](../docs/content/docs/connectors/table/pulsar.md) +- [flink-example: cookbooks, examples](https://github.com/streamnative/flink-example) +- [StreamNative Hub documentation](https://hub.streamnative.io/data-processing/pulsar-flink/1.15.0.1) + +> The legacy [pulsar-flink connector](https://github.com/streamnative/pulsar-flink) is deprecated. It will not be actively maintained and users are recommended to use this repository. diff --git a/.github/workflows/daily-end-to-end-test.yml b/.github/workflows/daily-end-to-end-test.yml new file mode 100644 index 0000000000000..4e8d492a4b992 --- /dev/null +++ b/.github/workflows/daily-end-to-end-test.yml @@ -0,0 +1,29 @@ +name: Daily End to End Test + +on: + schedule: + - cron: 0 1 * * * + workflow_dispatch: +jobs: + sync: + name: Run End to End test for pulsar + runs-on: ubuntu-latest + strategy: + matrix: + branch: + - develop + - release-sn-1.14 + - release-sn-1.15 + steps: + - name: Checkout the code + uses: actions/checkout@v2 + with: + ref: ${{ matrix.branch }} + - name: Set up Maven + uses: apache/pulsar-test-infra/setup-maven@master + with: + maven-version: 3.6.2 + - name: Run e2e Test + run: | + mvn clean install -DskipTests + mvn -pl 'flink-end-to-end-tests/flink-end-to-end-tests-pulsar' test diff --git a/.github/workflows/docs.sh b/.github/workflows/docs.sh new file mode 100755 index 0000000000000..fa0fd21bd6d07 --- /dev/null +++ b/.github/workflows/docs.sh @@ -0,0 +1,71 @@ +#!/usr/bin/env bash +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ +set -e + +mvn --version +java -version +javadoc -J-version + +# setup hugo +HUGO_REPO=https://github.com/gohugoio/hugo/releases/download/v0.80.0/hugo_extended_0.80.0_Linux-64bit.tar.gz +HUGO_ARTIFACT=hugo_extended_0.80.0_Linux-64bit.tar.gz +if ! curl --fail -OL $HUGO_REPO ; then + echo "Failed to download Hugo binary" + exit 1 +fi +tar -zxvf $HUGO_ARTIFACT +git submodule update --init --recursive +# generate docs into docs/target +./hugo -v --source docs --destination target +if [ $? -ne 0 ]; then + echo "Error building the docs" + exit 1 +fi + +# build Flink; required for Javadoc step +mvn clean install -B -DskipTests -Dfast -Pskip-webui-build + +# build java/scala docs +mkdir -p docs/target/api +mvn javadoc:aggregate -B \ + -Paggregate-scaladoc \ + -DadditionalJOption="-Xdoclint:none --allow-script-in-comments" \ + -Dmaven.javadoc.failOnError=false \ + -Dcheckstyle.skip=true \ + -Dspotless.check.skip=true \ + -Denforcer.skip=true \ + -Dheader="

Back to Flink Website

" +mv target/site/apidocs docs/target/api/java +pushd flink-scala +mvn scala:doc -B +mv target/site/scaladocs ../docs/target/api/scala +popd + +# build python docs +if [ -f ./flink-python/dev/lint-python.sh ]; then + # Just completely ignore sudo in conda. + unset SUDO_UID SUDO_GID SUDO_USER + + # build python docs + # disable the gateway, because otherwise it tries to find FLINK_HOME to access Java classes + PYFLINK_GATEWAY_DISABLED=1 ./flink-python/dev/lint-python.sh -i "sphinx" + + # move python docs + mv flink-python/docs/_build/html docs/target/api/python +fi diff --git a/.github/workflows/run-pr-check.yml b/.github/workflows/run-pr-check.yml new file mode 100644 index 0000000000000..c4b097e3dffb0 --- /dev/null +++ b/.github/workflows/run-pr-check.yml @@ -0,0 +1,23 @@ +name: Run Test for Pull Request + +on: + pull_request: + branches: [ develop, release-sn-** ] + +jobs: + test: + runs-on: ubuntu-latest + steps: + - name: Checkout the code + uses: actions/checkout@v2 + with: + ref: ${{ github.event.pull_request.head.sha }} + - name: Set up JDK 8 + uses: actions/setup-java@v2 + with: + java-version: '8' + cache: 'maven' + distribution: 'zulu' + - name: Run Unit Test and Install + run: | + mvn -ntp -pl 'flink-connectors/flink-connector-pulsar,flink-connectors/flink-sql-connector-pulsar' clean install diff --git a/docs/config.toml b/docs/config.toml index 8dfb6e3afbff6..ec0aec5e6de6a 100644 --- a/docs/config.toml +++ b/docs/config.toml @@ -14,7 +14,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -baseURL = '//nightlies.apache.org/flink/flink-docs-master' +baseURL = '//nightlies.apache.org/flink/flink-docs-release-1.15' languageCode = "en-us" title = "Apache Flink" enableGitInfo = false @@ -24,7 +24,7 @@ pygmentsUseClasses = true [params] # Flag whether this is a stable version or not. # Used for the quickstart page. - IsStable = false + IsStable = true # Flag to indicate whether an outdated warning should be shown. ShowOutDatedWarning = false @@ -34,14 +34,14 @@ pygmentsUseClasses = true # we change the version for the complete docs when forking of a release branch # etc. # The full version string as referenced in Maven (e.g. 1.2.1) - Version = "1.15-SNAPSHOT" + Version = "1.15.0" # For stable releases, leave the bugfix version out (e.g. 1.2). For snapshot # release this should be the same as the regular version - VersionTitle = "1.15-SNAPSHOT" + VersionTitle = "1.15" # The branch for this version of Apache Flink - Branch = "master" + Branch = "release-1.15" # The github repository for Apache Flink Repo = "//github.com/apache/flink" @@ -60,22 +60,23 @@ pygmentsUseClasses = true ZhDownloadPage = "//flink.apache.org/zh/downloads.html" - JavaDocs = "//nightlies.apache.org/flink/flink-docs-master/api/java/" + JavaDocs = "//nightlies.apache.org/flink/flink-docs-release-1.15/api/java/" - ScalaDocs = "//nightlies.apache.org/flink/flink-docs-master/api/scala/index.html#org.apache.flink.api.scala.package" + ScalaDocs = "//nightlies.apache.org/flink/flink-docs-release-1.15/api/scala/index.html#org.apache.flink.api.scala.package" - PyDocs = "//nightlies.apache.org/flink/flink-docs-master/api/python/" + PyDocs = "//nightlies.apache.org/flink/flink-docs-release-1.15/api/python/" # External links at the bottom # of the menu MenuLinks = [ ["Project Homepage", "//flink.apache.org"], - ["JavaDocs", "//nightlies.apache.org/flink/flink-docs-master/api/java/"], - ["ScalaDocs", "//nightlies.apache.org/flink/flink-docs-master/api/scala/index.html#org.apache.flink.api.scala.package"], - ["PyDocs", "//nightlies.apache.org/flink/flink-docs-master/api/python/"] + ["JavaDocs", "//nightlies.apache.org/flink/flink-docs-release-1.15/api/java/"], + ["ScalaDocs", "//nightlies.apache.org/flink/flink-docs-release-1.15/api/scala/index.html#org.apache.flink.api.scala.package/"], + ["PyDocs", "//nightlies.apache.org/flink/flink-docs-release-1.15/api/python/"] ] PreviousDocs = [ + ["1.15", "http://nightlies.apache.org/flink/flink-docs-release-1.15"], ["1.14", "http://nightlies.apache.org/flink/flink-docs-release-1.14"], ["1.13", "http://nightlies.apache.org/flink/flink-docs-release-1.13"], ["1.12", "http://nightlies.apache.org/flink/flink-docs-release-1.12"], diff --git a/docs/content.zh/docs/concepts/stateful-stream-processing.md b/docs/content.zh/docs/concepts/stateful-stream-processing.md index 011c96bb128ff..d4748b5b59e62 100644 --- a/docs/content.zh/docs/concepts/stateful-stream-processing.md +++ b/docs/content.zh/docs/concepts/stateful-stream-processing.md @@ -310,7 +310,10 @@ mechanism for this. Savepoints are similar to checkpoints except that they are **triggered by the user** and **don't automatically expire** when newer -checkpoints are completed. +checkpoints are completed. +To make proper use of savepoints, it's important to understand the differences between +[checkpoints]({{< ref "docs/ops/state/checkpoints" >}}) and [savepoints]({{< ref "docs/ops/state/savepoints" >}}) +which is described in [checkpoints vs. savepoints]({{< ref "docs/ops/state/checkpoints_vs_savepoints" >}}). {{< top >}} diff --git a/docs/content.zh/docs/concepts/time.md b/docs/content.zh/docs/concepts/time.md index 473aa0d8c1895..fbc1891ae08ac 100644 --- a/docs/content.zh/docs/concepts/time.md +++ b/docs/content.zh/docs/concepts/time.md @@ -84,8 +84,8 @@ one can refer to different notions of *time*: out-of-order or late events, or when reprocessing historic data. For example, an hourly event time window will contain all records that carry an event timestamp that falls into that hour, regardless of the order in which they - arrive, or when they are processed. (See the section on [late - events](#late-elements) for more information.) + arrive, or when they are processed. (See the section on [lateness](#lateness) + for more information.) Note that sometimes when event time programs are processing live data in real-time, they will use some *processing time* operations in order to diff --git a/docs/content.zh/docs/connectors/datastream/cassandra.md b/docs/content.zh/docs/connectors/datastream/cassandra.md index bddb9f2f1ba27..b13bfadf3bd7f 100644 --- a/docs/content.zh/docs/connectors/datastream/cassandra.md +++ b/docs/content.zh/docs/connectors/datastream/cassandra.md @@ -37,7 +37,7 @@ To use this connector, add the following dependency to your project: {{< artifact flink-connector-cassandra withScalaVersion >}} -Note that the streaming connectors are currently __NOT__ part of the binary distribution. See how to link with them for cluster execution [here]({{< ref "docs/dev/datastream/project-configuration" >}}). +Note that the streaming connectors are currently __NOT__ part of the binary distribution. See how to link with them for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). ## Installing Apache Cassandra There are multiple ways to bring up a Cassandra instance on local machine: diff --git a/docs/content.zh/docs/connectors/datastream/elasticsearch.md b/docs/content.zh/docs/connectors/datastream/elasticsearch.md index c97e85221afa8..aea933e17a8c8 100644 --- a/docs/content.zh/docs/connectors/datastream/elasticsearch.md +++ b/docs/content.zh/docs/connectors/datastream/elasticsearch.md @@ -52,7 +52,7 @@ under the License. 请注意,流连接器目前不是二进制发行版的一部分。 -有关如何将程序和用于集群执行的库一起打包,参考[此文档]({{< ref "docs/dev/datastream/project-configuration" >}}) +有关如何将程序和用于集群执行的库一起打包,参考[此文档]({{< ref "docs/dev/configuration/overview" >}})。 ## 安装 Elasticsearch @@ -132,7 +132,6 @@ private static IndexRequest createIndexRequest(String element) { return Requests.indexRequest() .index("my-index") - .type("my-type") .id(element) .source(json); } @@ -165,7 +164,7 @@ def createIndexRequest(element: (String)): IndexRequest = { "data" -> element.asInstanceOf[AnyRef] ) - Requests.indexRequest.index("my-index").`type`("my-type").source(mapAsJavaMap(json)) + Requests.indexRequest.index("my-index").source(mapAsJavaMap(json)) } ``` @@ -210,16 +209,13 @@ def createIndexRequest(element: (String)): IndexRequest = { ### Elasticsearch Sinks 和容错 -默认情况下,Flink Elasticsearch Sink 不会提供任何传递健壮性的保障。 -用户可以选择启用 Elasticsearch sink 的 at-least-once 语义。 - -通过启用 Flink checkpoint,Flink Elasticsearch Sink 可以保证至少一次将操作请求发送到 Elasticsearch 集群。 +通过启用 Flink checkpoint,Flink Elasticsearch Sink 保证至少一次将操作请求发送到 Elasticsearch 集群。 这是通过在进行 checkpoint 时等待 `BulkProcessor` 中所有挂起的操作请求来实现。 这有效地保证了在触发 checkpoint 之前所有的请求被 Elasticsearch 成功确认,然后继续处理发送到 sink 的记录。 关于 checkpoint 和容错的更多详细信息,请参见[容错文档]({{< ref "docs/learn-flink/fault_tolerance" >}})。 -要使用具有容错特性的 Elasticsearch Sinks,需要配置启用 at-least-once 分发并且在执行环境中启用作业拓扑的 checkpoint: +要使用具有容错特性的 Elasticsearch Sinks,需要在执行环境中启用作业拓扑的 checkpoint: {{< tabs "d00d1e93-4844-40d7-b0ec-9ec37e73145e" >}} {{< tab "Java" >}} @@ -276,6 +272,10 @@ val sinkBuilder = new Elasticsearch7SinkBuilder[String] {{< /tab >}} {{< /tabs >}} +

+Using UpdateRequests with deterministic ids and the upsert method it is possible to achieve exactly-once semantics in Elasticsearch when AT_LEAST_ONCE delivery is configured for the connector. +

+ ### 处理失败的 Elasticsearch 请求 Elasticsearch 操作请求可能由于多种原因而失败,包括节点队列容量暂时已满或者要被索引的文档格式错误。 @@ -373,7 +373,7 @@ checkpoint 会进行等待,直到 Elasticsearch 节点队列有足够的容量 ## 将 Elasticsearch 连接器打包到 Uber-Jar 中 建议构建一个包含所有依赖的 uber-jar (可执行的 jar),以便更好地执行你的 Flink 程序。 -(更多信息参见[此文档]({{< ref "docs/dev/datastream/project-configuration" >}}))。 +(更多信息参见[此文档]({{< ref "docs/dev/configuration/overview" >}}))。 或者,你可以将连接器的 jar 文件放入 Flink 的 `lib/` 目录下,使其在全局范围内可用,即可用于所有的作业。 diff --git a/docs/content.zh/docs/connectors/datastream/filesystem.md b/docs/content.zh/docs/connectors/datastream/filesystem.md index efd6d4c491cd5..b7514d17ef041 100644 --- a/docs/content.zh/docs/connectors/datastream/filesystem.md +++ b/docs/content.zh/docs/connectors/datastream/filesystem.md @@ -68,10 +68,10 @@ under the License. {{< tab "Java" >}} ```java // 从文件流中读取文件内容 -FileSource.forRecordStreamFormat(StreamFormat,Path...) +FileSource.forRecordStreamFormat(StreamFormat,Path...); // 从文件中一次读取一批记录 -FileSource.forBulkFileFormat(BulkFormat,Path...) +FileSource.forBulkFileFormat(BulkFormat,Path...); ``` {{< /tab >}} {{< /tabs >}} diff --git a/docs/content.zh/docs/connectors/datastream/formats/avro.md b/docs/content.zh/docs/connectors/datastream/formats/avro.md index 9e2190a2d5a8c..b8b4cea036eec 100644 --- a/docs/content.zh/docs/connectors/datastream/formats/avro.md +++ b/docs/content.zh/docs/connectors/datastream/formats/avro.md @@ -52,7 +52,7 @@ DataStream usersDS = env.createInput(users); 注意,`User` 是一个通过 Avro schema生成的 POJO 类。Flink 还允许选择 POJO 中字符串类型的键。例如: ```java -usersDS.keyBy("name") +usersDS.keyBy("name"); ``` diff --git a/docs/content.zh/docs/connectors/datastream/formats/hadoop.md b/docs/content.zh/docs/connectors/datastream/formats/hadoop.md index b3077c181aed8..bd19d3f0df3f6 100644 --- a/docs/content.zh/docs/connectors/datastream/formats/hadoop.md +++ b/docs/content.zh/docs/connectors/datastream/formats/hadoop.md @@ -106,7 +106,7 @@ Flink 为 Hadoop `OutputFormats` 提供了一个兼容性包装器。支持任 ```java // 获取我们希望发送的结果 -DataStream> hadoopResult = [...] +DataStream> hadoopResult = [...]; // 设置 the Hadoop TextOutputFormat。 HadoopOutputFormat hadoopOF = diff --git a/docs/content.zh/docs/connectors/datastream/formats/parquet.md b/docs/content.zh/docs/connectors/datastream/formats/parquet.md index 46828e50bd905..16b0249c72bb8 100644 --- a/docs/content.zh/docs/connectors/datastream/formats/parquet.md +++ b/docs/content.zh/docs/connectors/datastream/formats/parquet.md @@ -26,87 +26,114 @@ under the License. --> + + # Parquet format -Flink supports reading [Parquet](https://parquet.apache.org/) files, -producing {{< javadoc file="org/apache/flink/table/data/RowData.html" name="Flink RowData">}} and producing [Avro](https://avro.apache.org/) records. -To use the format you need to add the Flink Parquet dependency to your project: +Flink 支持读取 [Parquet](https://parquet.apache.org/) 文件并生成 {{< javadoc file="org/apache/flink/table/data/RowData.html" name="Flink RowData">}} 和 [Avro](https://avro.apache.org/) 记录。 +要使用 Parquet format,你需要将 flink-parquet 依赖添加到项目中: ```xml - org.apache.flink - flink-parquet - {{< version >}} + org.apache.flink + flink-parquet + {{< version >}} ``` -This format is compatible with the new Source that can be used in both batch and streaming modes. -Thus, you can use this format for two kinds of data: -- Bounded data -- Unbounded data: monitors a directory for new files that appear +要使用 Avro 格式,你需要将 parquet-avro 依赖添加到项目中: -## Flink RowData +```xml + + org.apache.parquet + parquet-avro + 1.12.2 + true + + + org.apache.hadoop + hadoop-client + + + it.unimi.dsi + fastutil + + + +``` + +此格式与新的 Source 兼容,可以同时在批和流模式下使用。 +因此,你可使用此格式处理以下两类数据: -#### Bounded data example +- 有界数据: 列出所有文件并全部读取。 +- 无界数据:监控目录中出现的新文件 -In this example, you will create a DataStream containing Parquet records as Flink RowDatas. The schema is projected to read only the specified fields ("f7", "f4" and "f99"). -Flink will read records in batches of 500 records. The first boolean parameter specifies that timestamp columns will be interpreted as UTC. -The second boolean instructs the application that the projected Parquet fields names are case-sensitive. -There is no watermark strategy defined as records do not contain event timestamps. +{{< hint info >}} +当你开启一个 File Source,会被默认为有界读取。 +如果你想在连续读取模式下使用 File Source,你必须额外调用 +`AbstractFileSource.AbstractFileSourceBuilder.monitorContinuously(Duration)`。 +{{< /hint >}} + +**Vectorized reader** ```java -final LogicalType[] fieldTypes = - new LogicalType[] { - new DoubleType(), new IntType(), new VarCharType() - }; +// Parquet rows are decoded in batches +FileSource.forBulkFileFormat(BulkFormat,Path...) +// Monitor the Paths to read data as unbounded data +FileSource.forBulkFileFormat(BulkFormat,Path...) +.monitorContinuously(Duration.ofMillis(5L)) +.build(); +``` -final ParquetColumnarRowInputFormat format = - new ParquetColumnarRowInputFormat<>( - new Configuration(), - RowType.of(fieldTypes, new String[] {"f7", "f4", "f99"}), - 500, - false, - true); -final FileSource source = - FileSource.forBulkFileFormat(format, /* Flink Path */) - .build(); -final DataStream stream = - env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); +**Avro Parquet reader** + +```java +// Parquet rows are decoded in batches +FileSource.forRecordStreamFormat(StreamFormat,Path...) +// Monitor the Paths to read data as unbounded data +FileSource.forRecordStreamFormat(StreamFormat,Path...) + .monitorContinuously(Duration.ofMillis(5L)) + .build(); ``` -#### Unbounded data example +{{< hint info >}} +下面的案例都是基于有界数据的。 +如果你想在连续读取模式下使用 File Source,你必须额外调用 +`AbstractFileSource.AbstractFileSourceBuilder.monitorContinuously(Duration)`。 +{{< /hint >}} -In this example, you will create a DataStream containing Parquet records as Flink RowDatas that will -infinitely grow as new files are added to the directory. It will monitor for new files each second. -The schema is projected to read only the specified fields ("f7", "f4" and "f99"). -Flink will read records in batches of 500 records. The first boolean parameter specifies that timestamp columns will be interpreted as UTC. -The second boolean instructs the application that the projected Parquet fields names are case-sensitive. -There is no watermark strategy defined as records do not contain event timestamps. + + +## Flink RowData + +在此示例中,你将创建由 Parquet 格式的记录构成的 Flink RowDatas DataStream。我们把 schema 信息映射为只读字段("f7"、"f4" 和 "f99")。 +每个批次读取 500 条记录。其中,第一个布尔类型的参数用来指定是否需要将时间戳列处理为 UTC。 +第二个布尔类型参数用来指定在进行 Parquet 字段映射时,是否要区分大小写。 +这里不需要水印策略,因为记录中不包含事件时间戳。 ```java final LogicalType[] fieldTypes = - new LogicalType[] { - new DoubleType(), new IntType(), new VarCharType() - }; + new LogicalType[] { + new DoubleType(), new IntType(), new VarCharType() + }; final ParquetColumnarRowInputFormat format = - new ParquetColumnarRowInputFormat<>( - new Configuration(), - RowType.of(fieldTypes, new String[] {"f7", "f4", "f99"}), - 500, - false, - true); + new ParquetColumnarRowInputFormat<>( + new Configuration(), + RowType.of(fieldTypes, new String[] {"f7", "f4", "f99"}), + 500, + false, + true); final FileSource source = - FileSource.forBulkFileFormat(format, /* Flink Path */) - .monitorContinuously(Duration.ofSeconds(1L)) - .build(); + FileSource.forBulkFileFormat(format, /* Flink Path */) + .build(); final DataStream stream = - env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); + env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); ``` ## Avro Records -Flink supports producing three types of Avro records by reading Parquet files: +Flink 支持三种方式来读取 Parquet 文件并创建 Avro records : - [Generic record](https://avro.apache.org/docs/1.10.0/api/java/index.html) - [Specific record](https://avro.apache.org/docs/1.10.0/api/java/index.html) @@ -114,97 +141,62 @@ Flink supports producing three types of Avro records by reading Parquet files: ### Generic record -Avro schemas are defined using JSON. You can get more information about Avro schemas and types from the [Avro specification](https://avro.apache.org/docs/1.10.0/spec.html). -This example uses an Avro schema example similar to the one described in the [official Avro tutorial](https://avro.apache.org/docs/1.10.0/gettingstartedjava.html): +使用 JSON 定义 Avro schemas。你可以从 [Avro specification](https://avro.apache.org/docs/1.10.0/spec.html) 获取更多关于 Avro schemas 和类型的信息。 +此示例使用了一个在 [official Avro tutorial](https://avro.apache.org/docs/1.10.0/gettingstartedjava.html) 中描述的示例相似的 Avro schema: ```json lines {"namespace": "example.avro", - "type": "record", - "name": "User", - "fields": [ - {"name": "name", "type": "string"}, - {"name": "favoriteNumber", "type": ["int", "null"]}, - {"name": "favoriteColor", "type": ["string", "null"]} - ] + "type": "record", + "name": "User", + "fields": [ + {"name": "name", "type": "string"}, + {"name": "favoriteNumber", "type": ["int", "null"]}, + {"name": "favoriteColor", "type": ["string", "null"]} + ] } ``` +这个 schema 定义了一个具有三个属性的的 user 记录:name,favoriteNumber 和 favoriteColor。你可以 +在 [record specification](https://avro.apache.org/docs/1.10.0/spec.html#schema_record) 找到更多关于如何定义 Avro schema 的详细信息。 -This schema defines a record representing a user with three fields: name, favoriteNumber, and favoriteColor. You can find more details at [record specification](https://avro.apache.org/docs/1.10.0/spec.html#schema_record) for how to define an Avro schema. - -#### Bounded data example - -In this example, you will create a DataStream containing Parquet records as Avro Generic records. -It will parse the Avro schema based on the JSON string. There are many other ways to parse a schema, e.g. from java.io.File or java.io.InputStream. Please refer to [Avro Schema](https://avro.apache.org/docs/1.10.0/api/java/org/apache/avro/Schema.html) for details. -After that, you will create an `AvroParquetRecordFormat` via `AvroParquetReaders` for Avro Generic records. +在此示例中,你将创建包含由 Avro Generic records 格式构成的 Parquet records 的 DataStream。 +Flink 会基于 JSON 字符串解析 Avro schema。也有很多其他的方式解析 schema,例如基于 java.io.File 或 java.io.InputStream。 +请参考 [Avro Schema](https://avro.apache.org/docs/1.10.0/api/java/org/apache/avro/Schema.html) 以获取更多详细信息。 +然后,你可以通过 `AvroParquetReaders` 为 Avro Generic 记录创建 `AvroParquetRecordFormat`。 ```java -// parsing avro schema +// 解析 avro schema final Schema schema = new Schema.Parser() - .parse( - "{\"type\": \"record\", " - + "\"name\": \"User\", " - + "\"fields\": [\n" - + " {\"name\": \"name\", \"type\": \"string\" },\n" - + " {\"name\": \"favoriteNumber\", \"type\": [\"int\", \"null\"] },\n" - + " {\"name\": \"favoriteColor\", \"type\": [\"string\", \"null\"] }\n" - + " ]\n" - + " }"); + .parse( + "{\"type\": \"record\", " + + "\"name\": \"User\", " + + "\"fields\": [\n" + + " {\"name\": \"name\", \"type\": \"string\" },\n" + + " {\"name\": \"favoriteNumber\", \"type\": [\"int\", \"null\"] },\n" + + " {\"name\": \"favoriteColor\", \"type\": [\"string\", \"null\"] }\n" + + " ]\n" + + " }"); final FileSource source = FileSource.forRecordStreamFormat( - AvroParquetReaders.forGenericRecord(schema), /* Flink Path */) + AvroParquetReaders.forGenericRecord(schema), /* Flink Path */) .build(); final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.enableCheckpointing(10L); - -final DataStream stream = - env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); -``` -#### Unbounded data example - -This example is similar to the bounded batch example. The application monitors for new files every second -and reads Avro Generic records from Parquet files infinitely as new files are added to the directory. -```java -// parsing avro schema -final Schema schema = - new Schema.Parser() - .parse( - "{\"type\": \"record\", " - + "\"name\": \"User\", " - + "\"fields\": [\n" - + " {\"name\": \"name\", \"type\": \"string\" },\n" - + " {\"name\": \"favoriteNumber\", \"type\": [\"int\", \"null\"] },\n" - + " {\"name\": \"favoriteColor\", \"type\": [\"string\", \"null\"] }\n" - + " ]\n" - + " }"); - -final FileSource source = - FileSource.forRecordStreamFormat( - AvroParquetReaders.forGenericRecord(schema), /* Flink Path */) - .monitorContinuously(Duration.ofSeconds(1L)) - .build(); - -final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.enableCheckpointing(10L); - final DataStream stream = env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); ``` ### Specific record -Based on the previously defined schema, you can generate classes by leveraging Avro code generation. -Once the classes have been generated, there is no need to use the schema directly in your programs. -You can either use `avro-tools.jar` to generate code manually or you could use the Avro Maven plugin to perform -code generation on any .avsc files present in the configured source directory. Please refer to -[Avro Getting Started](https://avro.apache.org/docs/1.10.0/gettingstartedjava.html) for more information. - -#### Bounded data example +基于之前定义的 schema,你可以通过利用 Avro 代码生成来生成类。 +一旦生成了类,就不需要在程序中直接使用 schema。 +你可以使用 `avro-tools.jar` 手动生成代码,也可以直接使用 Avro Maven 插件对配置的源目录中的任何 .avsc 文件执行代码生成。 +请参考 [Avro Getting Started](https://avro.apache.org/docs/1.10.0/gettingstartedjava.html) 获取更多信息。 -This example uses the example schema [testdata.avsc](https://github.com/apache/flink/blob/master/flink-formats/flink-parquet/src/test/resources/avro/testdata.avsc): +此示例使用了样例 schema [testdata.avsc](https://github.com/apache/flink/blob/master/flink-formats/flink-parquet/src/test/resources/avro/testdata.avsc): ```json lines [ @@ -222,17 +214,17 @@ This example uses the example schema [testdata.avsc](https://github.com/apache/f ] ``` -You will use the Avro Maven plugin to generate the `Address` Java class: +你可以使用 Avro Maven plugin 生成 `Address` Java 类。 ```java @org.apache.avro.specific.AvroGenerated public class Address extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { - // generated code... + // 生成的代码... } ``` -You will create an `AvroParquetRecordFormat` via `AvroParquetReaders` for Avro Specific record -and then create a DataStream containing Parquet records as Avro Specific records. +你可以通过 `AvroParquetReaders` 为 Avro Specific 记录创建 `AvroParquetRecordFormat`, +然后创建一个包含由 Avro Specific records 格式构成的 Parquet records 的 DateStream。 ```java final FileSource source = @@ -247,36 +239,13 @@ final DataStream stream = env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); ``` -#### Unbounded data example - -This example, similar to the bounded batch example, uses the same generated Address Java class -and monitors for the new files every second to read Avro Specific records from Parquet files -infinitely as new files are added to the directory. - -```java -final FileSource source = - FileSource.forRecordStreamFormat( - AvroParquetReaders.forSpecificRecord(Address.class), /* Flink Path */) - .monitorContinuously(Duration.ofSeconds(1L)) - .build(); - -final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.enableCheckpointing(10L); - -final DataStream stream = - env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); -``` - ### Reflect record -Beyond Avro Generic and Specific record that requires a predefined Avro schema, -Flink also supports creating a DataStream from Parquet files based on existing Java POJO classes. -In this case, Avro will use Java reflection to generate schemas and protocols for these POJO classes. -Java types are mapped to Avro schemas, please refer to the [Avro reflect](https://avro.apache.org/docs/1.10.0/api/java/index.html) documentation for more details. - -#### Bounded data example +除了需要预定义 Avro Generic 和 Specific 记录, Flink 还支持基于现有 Java POJO 类从 Parquet 文件创建 DateStream。 +在这种场景中,Avro 会使用 Java 反射为这些 POJO 类生成 schema 和协议。 +请参考 [Avro reflect](https://avro.apache.org/docs/1.10.0/api/java/index.html) 文档获取更多关于 Java 类型到 Avro schemas 映射的详细信息。 -This example uses a simple Java POJO class [Datum](https://github.com/apache/flink/blob/master/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/avro/Datum.java): +本例使用了一个简单的 Java POJO 类 [Datum](https://github.com/apache/flink/blob/master/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/avro/Datum.java): ```java public class Datum implements Serializable { @@ -313,8 +282,8 @@ public class Datum implements Serializable { } ``` -You will create an `AvroParquetRecordFormat` via `AvroParquetReaders` for Avro Reflect record -and then create a DataStream containing Parquet records as Avro Reflect records. +你可以通过 `AvroParquetReaders` 为 Avro Reflect 记录创建一个 `AvroParquetRecordFormat`, +然后创建一个包含由 Avro Reflect records 格式构成的 Parquet records 的 DateStream。 ```java final FileSource source = @@ -329,22 +298,90 @@ final DataStream stream = env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); ``` -#### Unbounded data example +### 使用 Parquet files 必备条件 -This example, similar to the bounded batch example, uses the same POJO Java class `Datum` -and monitors for the new files every second to read Avro Reflect records from Parquet files -infinitely as new files are added to the directory. +为了支持读取 Avro Reflect 数据,Parquet 文件必须包含特定的 meta 信息。为了生成 Parquet 数据,Avro schema 信息中必须包含 namespace, +以便让程序在反射执行过程中能确定唯一的 Java Class 对象。 + +下面的案例展示了上文中的 User 对象的 schema 信息。但是当前案例包含了一个指定文件目录的 namespace(当前案例下的包路径),反射过程中可以找到对应的 User 类。 + +```java +// avro schema with namespace +final String schema = + "{\"type\": \"record\", " + + "\"name\": \"User\", " + + "\"namespace\": \"org.apache.flink.formats.parquet.avro\", " + + "\"fields\": [\n" + + " {\"name\": \"name\", \"type\": \"string\" },\n" + + " {\"name\": \"favoriteNumber\", \"type\": [\"int\", \"null\"] },\n" + + " {\"name\": \"favoriteColor\", \"type\": [\"string\", \"null\"] }\n" + + " ]\n" + + " }"; +``` + +由上述 scheme 信息创建的 Parquet 文件包含以下 meta 信息: + +```text +creator: parquet-mr version 1.12.2 (build 77e30c8093386ec52c3cfa6c34b7ef3321322c94) +extra: parquet.avro.schema = +{"type":"record","name":"User","namespace":"org.apache.flink.formats.parquet.avro","fields":[{"name":"name","type":"string"},{"name":"favoriteNumber","type":["int","null"]},{"name":"favoriteColor","type":["string","null"]}]} +extra: writer.model.name = avro + +file schema: org.apache.flink.formats.parquet.avro.User +-------------------------------------------------------------------------------- +name: REQUIRED BINARY L:STRING R:0 D:0 +favoriteNumber: OPTIONAL INT32 R:0 D:1 +favoriteColor: OPTIONAL BINARY L:STRING R:0 D:1 + +row group 1: RC:3 TS:143 OFFSET:4 +-------------------------------------------------------------------------------- +name: BINARY UNCOMPRESSED DO:0 FPO:4 SZ:47/47/1.00 VC:3 ENC:PLAIN,BIT_PACKED ST:[min: Jack, max: Tom, num_nulls: 0] +favoriteNumber: INT32 UNCOMPRESSED DO:0 FPO:51 SZ:41/41/1.00 VC:3 ENC:RLE,PLAIN,BIT_PACKED ST:[min: 1, max: 3, num_nulls: 0] +favoriteColor: BINARY UNCOMPRESSED DO:0 FPO:92 SZ:55/55/1.00 VC:3 ENC:RLE,PLAIN,BIT_PACKED ST:[min: green, max: yellow, num_nulls: 0] + +``` + +使用包 `org.apache.flink.formats.parquet.avro` 路径下已定义的 User 类: + +```java +public class User { + private String name; + private Integer favoriteNumber; + private String favoriteColor; + + public User() {} + + public User(String name, Integer favoriteNumber, String favoriteColor) { + this.name = name; + this.favoriteNumber = favoriteNumber; + this.favoriteColor = favoriteColor; + } + + public String getName() { + return name; + } + + public Integer getFavoriteNumber() { + return favoriteNumber; + } + + public String getFavoriteColor() { + return favoriteColor; + } +} + +``` + +你可以通过下面的程序读取类型为 User 的 Avro Reflect records: ```java final FileSource source = FileSource.forRecordStreamFormat( - AvroParquetReaders.forReflectRecord(Datum.class), /* Flink Path */) - .monitorContinuously(Duration.ofSeconds(1L)) + AvroParquetReaders.forReflectRecord(User.class), /* Flink Path */) .build(); - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.enableCheckpointing(10L); - + final DataStream stream = env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); ``` diff --git a/docs/content.zh/docs/connectors/datastream/jdbc.md b/docs/content.zh/docs/connectors/datastream/jdbc.md index 50fde2ea1d339..31e4f182fcdf4 100644 --- a/docs/content.zh/docs/connectors/datastream/jdbc.md +++ b/docs/content.zh/docs/connectors/datastream/jdbc.md @@ -32,7 +32,7 @@ under the License. {{< artifact flink-connector-jdbc >}} -注意该连接器目前还 __不是__ 二进制发行版的一部分,如何在集群中运行请参考 [这里]({{< ref "docs/dev/datastream/project-configuration" >}})。 +注意该连接器目前还 __不是__ 二进制发行版的一部分,如何在集群中运行请参考 [这里]({{< ref "docs/dev/configuration/overview" >}})。 已创建的 JDBC Sink 能够保证至少一次的语义。 更有效的精确执行一次可以通过 upsert 语句或幂等更新实现。 diff --git a/docs/content.zh/docs/connectors/datastream/kafka.md b/docs/content.zh/docs/connectors/datastream/kafka.md index 3908f19d747a9..39010fbf791c2 100644 --- a/docs/content.zh/docs/connectors/datastream/kafka.md +++ b/docs/content.zh/docs/connectors/datastream/kafka.md @@ -43,7 +43,7 @@ Apache Flink 集成了通用的 Kafka 连接器,它会尽力与 Kafka client {{< artifact flink-connector-base >}} Flink 目前的流连接器还不是二进制发行版的一部分。 -[在此处]({{< ref "docs/dev/datastream/project-configuration" >}})可以了解到如何链接它们,从而在集群中运行。 +[在此处]({{< ref "docs/dev/configuration/overview" >}})可以了解到如何链接它们,从而在集群中运行。 ## Kafka Source {{< hint info >}} @@ -74,18 +74,18 @@ env.fromSource(source, WatermarkStrategy.noWatermarks(), "Kafka Source"); Kafka Source 提供了 3 种 Topic / Partition 的订阅方式: - Topic 列表,订阅 Topic 列表中所有 Partition 的消息: ```java - KafkaSource.builder().setTopics("topic-a", "topic-b") + KafkaSource.builder().setTopics("topic-a", "topic-b"); ``` - 正则表达式匹配,订阅与正则表达式所匹配的 Topic 下的所有 Partition: ```java - KafkaSource.builder().setTopicPattern("topic.*") + KafkaSource.builder().setTopicPattern("topic.*"); ``` - Partition 列表,订阅指定的 Partition: ```java final HashSet partitionSet = new HashSet<>(Arrays.asList( new TopicPartition("topic-a", 0), // Partition 0 of topic "topic-a" new TopicPartition("topic-b", 5))); // Partition 5 of topic "topic-b" - KafkaSource.builder().setPartitions(partitionSet) + KafkaSource.builder().setPartitions(partitionSet); ``` ### 消息解析 代码中需要提供一个反序列化器(Deserializer)来对 Kafka 的消息进行解析。 @@ -102,7 +102,7 @@ Kafka Source 提供了 3 种 Topic / Partition 的订阅方式: import org.apache.kafka.common.serialization.StringDeserializer; KafkaSource.builder() - .setDeserializer(KafkaRecordDeserializationSchema.valueOnly(StringSerializer.class)); + .setDeserializer(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class)); ``` ### 起始消费位点 @@ -119,7 +119,7 @@ KafkaSource.builder() // 从最早位点开始消费 .setStartingOffsets(OffsetsInitializer.earliest()) // 从最末尾位点开始消费 - .setStartingOffsets(OffsetsInitializer.latest()) + .setStartingOffsets(OffsetsInitializer.latest()); ``` 如果内置的初始化器不能满足需求,也可以实现自定义的位点初始化器(```OffsetsInitializer```)。 @@ -153,7 +153,7 @@ Kafka consumer 的配置可以参考 [Apache Kafka 文档](http://kafka.apache.o ```java KafkaSource.builder() .setProperty("sasl.mechanism", "PLAIN") - .setProperty("sasl.jaas.config", "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"username\" password=\"password\";") + .setProperty("sasl.jaas.config", "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"username\" password=\"password\";"); ``` ### 动态分区检查 @@ -162,7 +162,7 @@ KafkaSource.builder() ```java KafkaSource.builder() - .setProperty("partition.discovery.interval.ms", "10000") // 每 10 秒检查一次新分区 + .setProperty("partition.discovery.interval.ms", "10000"); // 每 10 秒检查一次新分区 ``` {{< hint warning >}} 分区检查功能默认**不开启**。需要显式地设置分区检查间隔才能启用此功能。 @@ -172,7 +172,7 @@ KafkaSource.builder() 默认情况下,Kafka Source 使用 Kafka 消息中的时间戳作为事件时间。您可以定义自己的水印策略(Watermark Strategy) 以从消息中提取事件时间,并向下游发送水印: ```java -env.fromSource(kafkaSource, new CustomWatermarkStrategy(), "Kafka Source With Custom Watermark Strategy") +env.fromSource(kafkaSource, new CustomWatermarkStrategy(), "Kafka Source With Custom Watermark Strategy"); ``` [这篇文档]({{< ref "docs/dev/datastream/event-time/generating_watermarks.md" >}})描述了如何自定义水印策略(```WatermarkStrategy```)。 @@ -314,7 +314,7 @@ Kafka sink 提供了构建类来创建 ```KafkaSink``` 的实例。以下代码 topic: ```java -DataStream stream = ... +DataStream stream = ...; KafkaSink sink = KafkaSink.builder() .setBootstrapServers(brokers) diff --git a/docs/content.zh/docs/connectors/datastream/nifi.md b/docs/content.zh/docs/connectors/datastream/nifi.md index d8e53e6ec59a4..a67b9cbb666e9 100644 --- a/docs/content.zh/docs/connectors/datastream/nifi.md +++ b/docs/content.zh/docs/connectors/datastream/nifi.md @@ -35,7 +35,7 @@ The NiFi connector is deprecated and will be removed with Flink 1.16. {{< artifact flink-connector-nifi >}} -注意这些连接器目前还没有包含在二进制发行版中。添加依赖、打包配置以及集群运行的相关信息请参考 [这里]({{< ref "docs/dev/datastream/project-configuration" >}})。 +注意这些连接器目前还没有包含在二进制发行版中。添加依赖、打包配置以及集群运行的相关信息请参考 [这里]({{< ref "docs/dev/configuration/overview" >}})。 #### 安装 Apache NiFi diff --git a/docs/content.zh/docs/connectors/datastream/pubsub.md b/docs/content.zh/docs/connectors/datastream/pubsub.md index 18225ca674b55..04e2a01ea6b34 100644 --- a/docs/content.zh/docs/connectors/datastream/pubsub.md +++ b/docs/content.zh/docs/connectors/datastream/pubsub.md @@ -34,7 +34,7 @@ under the License. 注意:此连接器最近才加到 Flink 里,还未接受广泛测试。

-注意连接器目前还不是二进制发行版的一部分,添加依赖、打包配置以及集群运行信息请参考[这里]({{< ref "docs/dev/datastream/project-configuration" >}}) +注意连接器目前还不是二进制发行版的一部分,添加依赖、打包配置以及集群运行信息请参考[这里]({{< ref "docs/dev/configuration/overview" >}}) ## Consuming or Producing PubSubMessages diff --git a/docs/content.zh/docs/connectors/datastream/pulsar.md b/docs/content.zh/docs/connectors/datastream/pulsar.md index 1779c729fd7eb..6c428f2a4166a 100644 --- a/docs/content.zh/docs/connectors/datastream/pulsar.md +++ b/docs/content.zh/docs/connectors/datastream/pulsar.md @@ -24,31 +24,27 @@ under the License. # Apache Pulsar 连接器 -Flink 当前只提供 [Apache Pulsar](https://pulsar.apache.org) 数据源,用户可以使用它从 Pulsar 读取数据,并保证每条数据只被处理一次。 +Flink 当前提供 [Apache Pulsar](https://pulsar.apache.org) Source 和 Sink 连接器,用户可以使用它从 Pulsar 读取数据,并保证每条数据只被处理一次。 ## 添加依赖 -连接器当前支持 Pulsar 2.7.0 之后的版本,但是连接器使用到了 Pulsar 的[事务机制](https://pulsar.apache.org/docs/en/txn-what/),建议在 Pulsar 2.8.0 -及其之后的版本上使用连接器进行数据读取。 +Pulsar Source 当前支持 Pulsar 2.8.1 之后的版本,但是 Pulsar Source 使用到了 Pulsar 的[事务机制](https://pulsar.apache.org/docs/zh-CN/txn-what/),建议在 Pulsar 2.9.2 及其之后的版本上使用 Pulsar Source 进行数据读取。 如果想要了解更多关于 Pulsar API 兼容性设计,可以阅读文档 [PIP-72](https://github.com/apache/pulsar/wiki/PIP-72%3A-Introduce-Pulsar-Interface-Taxonomy%3A-Audience-and-Stability-Classification)。 {{< artifact flink-connector-pulsar >}} -Flink 的流连接器并不会放到发行文件里面一同发布,阅读[此文档]({{< ref "docs/dev/datastream/project-configuration" >}}),了解如何将连接器添加到集群实例内。 +Flink 的流连接器并不会放到发行文件里面一同发布,阅读[此文档]({{< ref "docs/dev/configuration/overview" >}}),了解如何将连接器添加到集群实例内。 -## Pulsar 数据源 +## Pulsar Source {{< hint info >}} -Pulsar 数据源基于 Flink 最新的[批流一体 API]({{< ref "docs/dev/datastream/sources.md" >}}) 进行开发。 - -如果要想使用基于旧版的 `SourceFunction` 实现的 Pulsar 数据源,或者是项目的 Flink 版本低于 1.14,可以使用 StreamNative 单独维护的 [pulsar-flink](https://github.com/streamnative/pulsar-flink)。 +Pulsar Source 基于 Flink 最新的[批流一体 API]({{< ref "docs/dev/datastream/sources.md" >}}) 进行开发。 {{< /hint >}} ### 使用示例 -Pulsar 数据源提供了 builder 类来构造数据源实例。下面的代码实例使用 builder 类创建的数据源会从 topic "persistent://public/default/my-topic" 的数据开始端进行消费。 -连接器使用了 **Exclusive**(独占)的订阅方式消费消息,订阅名称为 `my-subscription`,并把消息体的二进制字节流以 UTF-8 的方式编码为字符串。 +Pulsar Source 提供了 builder 类来构造 `PulsarSource` 实例。下面的代码实例使用 builder 类创建的实例会从 “persistent://public/default/my-topic” 的数据开始端进行消费。对应的 Pulsar Source 使用了 **Exclusive**(独占)的订阅方式消费消息,订阅名称为 `my-subscription`,并把消息体的二进制字节流以 UTF-8 的方式编码为字符串。 ```java PulsarSource pulsarSource = PulsarSource.builder() @@ -64,222 +60,229 @@ PulsarSource pulsarSource = PulsarSource.builder() env.fromSource(source, WatermarkStrategy.noWatermarks(), "Pulsar Source"); ``` -如果使用构造类构造 Pulsar 数据源,一定要提供下面几个属性: +如果使用构造类构造 `PulsarSource`,一定要提供下面几个属性: -- Pulsar 数据消费的地址,使用 `setServiceUrl(String)` 方法提供 -- Pulsar HTTP 管理地址,使用 `setAdminUrl(String)` 方法提供 -- Pulsar 订阅名称,使用 `setSubscriptionName(String)` 方法提供 -- 需要消费的 topic 或者是 topic 下面的分区,详见[指定消费的 Topic 或者 Topic 分区](#指定消费的-topic-或者-topic-分区) -- 解码 Pulsar 消息的反序列化器,详见[反序列化器](#反序列化器) +- Pulsar 数据消费的地址,使用 `setServiceUrl(String)` 方法提供。 +- Pulsar HTTP 管理地址,使用 `setAdminUrl(String)` 方法提供。 +- Pulsar 订阅名称,使用 `setSubscriptionName(String)` 方法提供。 +- 需要消费的 Topic 或者是 Topic 下面的分区,详见[指定消费的 Topic 或者 Topic 分区](#指定消费的-topic-或者-topic-分区)。 +- 解码 Pulsar 消息的反序列化器,详见[反序列化器](#反序列化器)。 ### 指定消费的 Topic 或者 Topic 分区 -Pulsar 数据源提供了两种订阅 topic 或 topic 分区的方式。 +Pulsar Source 提供了两种订阅 Topic 或 Topic 分区的方式。 - Topic 列表,从这个 Topic 的所有分区上消费消息,例如: ```java - PulsarSource.builder().setTopics("some-topic1", "some-topic2") + PulsarSource.builder().setTopics("some-topic1", "some-topic2"); // 从 topic "topic-a" 的 0 和 1 分区上消费 - PulsarSource.builder().setTopics("topic-a-partition-0", "topic-a-partition-2") + PulsarSource.builder().setTopics("topic-a-partition-0", "topic-a-partition-2"); ``` -- Topic 正则,连接器使用给定的正则表达式匹配出所有合规的 topic,例如: +- Topic 正则,Pulsar Source 使用给定的正则表达式匹配出所有合规的 Topic,例如: ```java - PulsarSource.builder().setTopicPattern("topic-*") + PulsarSource.builder().setTopicPattern("topic-*"); ``` #### Topic 名称简写 -从 Pulsar 2.0 之后,完整的 topic 名称格式为 `{persistent|non-persistent}://租户/命名空间/topic`。 -但是连接器不需要提供 topic 名称的完整定义,因为 topic 类型、租户、命名空间都设置了默认值。 +从 Pulsar 2.0 之后,完整的 Topic 名称格式为 `{persistent|non-persistent}://租户/命名空间/topic`。但是 Pulsar Source 不需要提供 Topic 名称的完整定义,因为 Topic 类型、租户、命名空间都设置了默认值。 -Topic 属性 | 默认值 -:------------|:------- -topic 类型 | `persistent` -租户 | `public` -命名空间 | `default` +| Topic 属性 | 默认值 | +|:---------|:-------------| +| Topic 类型 | `persistent` | +| 租户 | `public` | +| 命名空间 | `default` | -下面的表格提供了当前 Pulsar 支持的简写方式: +下面的表格提供了当前 Pulsar Topic 支持的简写方式: -topic 名称简写 | 翻译后的 topic 名称 -:----------------|:--------------------- -`my-topic` | `persistent://public/default/my-topic` -`my-tenant/my-namespace/my-topic` | `persistent://my-tenant/my-namespace/my-topic` +| Topic 名称简写 | 翻译后的 Topic 名称 | +|:----------------------------------|:-----------------------------------------------| +| `my-topic` | `persistent://public/default/my-topic` | +| `my-tenant/my-namespace/my-topic` | `persistent://my-tenant/my-namespace/my-topic` | {{< hint warning >}} -对于 non-persistent(非持久化) topic,连接器不支持简写名称。所以无法将 `non-persistent://public/default/my-topic` 简写成 `non-persistent://my-topic`。 +对于 Non-persistent(非持久化)Topic,Pulsar Source 不支持简写名称。所以无法将 `non-persistent://public/default/my-topic` 简写成 `non-persistent://my-topic`。 {{< /hint >}} #### Pulsar Topic 层次结构 对于 Pulsar 而言,Topic 分区也是一种 Topic。Pulsar 会将一个有分区的 Topic 在内部按照分区的大小拆分成等量的无分区 Topic。 -例如,在 Pulsar 的 `sample` 租户下面的 `flink` 命名空间里面创建了一个有 3 个分区的 topic,给它起名为 `simple-string`。 -可以在 Pulsar 上看到如下的 topic 列表: +由于 Pulsar 内部的分区实际实现为一个 Topic,我们将用“分区”来指代“仅有一个分区的 Topic(Non-partitioned Topic)”和“具有多个分区的 Topic 下属的分区”。 + +例如,在 Pulsar 的 `sample` 租户下面的 `flink` 命名空间里面创建了一个有 3 个分区的 Topic,给它起名为 `simple-string`。可以在 Pulsar 上看到如下的 Topic 列表: -Topic 名称 | 是否分区 -:--------- | :---------- -`persistent://sample/flink/simple-string` | 是 -`persistent://sample/flink/simple-string-partition-0` | 否 -`persistent://sample/flink/simple-string-partition-1` | 否 -`persistent://sample/flink/simple-string-partition-2` | 否 +| Topic 名称 | 是否分区 | +|:------------------------------------------------------|:-----| +| `persistent://sample/flink/simple-string` | 是 | +| `persistent://sample/flink/simple-string-partition-0` | 否 | +| `persistent://sample/flink/simple-string-partition-1` | 否 | +| `persistent://sample/flink/simple-string-partition-2` | 否 | -这意味着,用户可以用上面的子 topic 去直接消费分区里面的数据,不需要再去基于上层的父 topic 去消费全部分区的数据。 -例如:使用 `PulsarSource.builder().setTopics("sample/flink/simple-string-partition-1", "sample/flink/simple-string-partition-2")` 将会只消费 topic `sample/flink/simple-string` 上面的分区 1 和 2 里面的消息。 +这意味着,用户可以用上面的子 Topic 去直接消费分区里面的数据,不需要再去基于上层的父 Topic 去消费全部分区的数据。例如:使用 `PulsarSource.builder().setTopics("sample/flink/simple-string-partition-1", "sample/flink/simple-string-partition-2")` 将会只消费 Topic `sample/flink/simple-string` 分区 1 和 2 里面的消息。 #### 配置 Topic 正则表达式 -前面提到了 Pulsar topic 有 `persistent`、`non-persistent` 两种类型,使用正则表达式消费数据的时候,连接器会尝试从正则表达式里面解析出消息的类型。 -例如:`PulsarSource.builder().setTopicPattern("non-persistent://my-topic*")` 会解析出 `non-persistent` 这个 topic 类型。 -如果用户使用 topic 名称简写的方式,连接器会使用默认的消息类型 `persistent`。 +前面提到了 Pulsar Topic 有 `persistent`、`non-persistent` 两种类型,使用正则表达式消费数据的时候,Pulsar Source 会尝试从正则表达式里面解析出消息的类型。例如:`PulsarSource.builder().setTopicPattern("non-persistent://my-topic*")` 会解析出 `non-persistent` 这个 Topic 类型。如果用户使用 Topic 名称简写的方式,Pulsar Source 会使用默认的消息类型 `persistent`。 -如果想用正则去消费 `persistent` 和 `non-persistent` 类型的 topic,需要使用 `RegexSubscriptionMode` 定义 topic 类型,例如:`setTopicPattern("topic-*", RegexSubscriptionMode.AllTopics)`。 +如果想用正则去消费 `persistent` 和 `non-persistent` 类型的 Topic,需要使用 `RegexSubscriptionMode` 定义 Topic 类型,例如:`setTopicPattern("topic-*", RegexSubscriptionMode.AllTopics)`。 ### 反序列化器 -反序列化器用于解析 Pulsar 消息,连接器使用 `PulsarDeserializationSchema` 来定义反序列化器。 -用户可以在 builder 类中使用 `setDeserializationSchema(PulsarDeserializationSchema)` 方法配置反序列化器,它会解析 Pulsar 的 `Message` 实例。 +反序列化器用于解析 Pulsar 消息,Pulsar Source 使用 `PulsarDeserializationSchema` 来定义反序列化器。用户可以在 builder 类中使用 `setDeserializationSchema(PulsarDeserializationSchema)` 方法配置反序列化器。 -如果用户只关心消息体的二进制字节流,并不需要其他属性来解析数据。可以直接使用预定义的 `PulsarDeserializationSchema`。Pulsar 连接器里面提供了 3 种预定义好的反序列化器。 +如果用户只关心消息体的二进制字节流,并不需要其他属性来解析数据。可以直接使用预定义的 `PulsarDeserializationSchema`。Pulsar Source里面提供了 3 种预定义的反序列化器。 -- 使用 Pulsar 的 [Schema](https://pulsar.apache.org/docs/en/schema-understand/) 解析消息。 +- 使用 Pulsar 的 [Schema](https://pulsar.apache.org/docs/zh-CN/schema-understand/) 解析消息。 ```java // 基础数据类型 - PulsarDeserializationSchema.pulsarSchema(Schema) + PulsarDeserializationSchema.pulsarSchema(Schema); // 结构类型 (JSON, Protobuf, Avro, etc.) - PulsarDeserializationSchema.pulsarSchema(Schema, Class) + PulsarDeserializationSchema.pulsarSchema(Schema, Class); // 键值对类型 - PulsarDeserializationSchema.pulsarSchema(Schema, Class, Class) + PulsarDeserializationSchema.pulsarSchema(Schema, Class, Class); ``` - 使用 Flink 的 `DeserializationSchema` 解析消息。 ```java - PulsarDeserializationSchema.flinkSchema(DeserializationSchema) + PulsarDeserializationSchema.flinkSchema(DeserializationSchema); ``` - 使用 Flink 的 `TypeInformation` 解析消息。 ```java - PulsarDeserializationSchema.flinkTypeInfo(TypeInformation, ExecutionConfig) + PulsarDeserializationSchema.flinkTypeInfo(TypeInformation, ExecutionConfig); ``` -Pulsar 的 `Message` 包含了很多 [额外的属性](https://pulsar.apache.org/docs/zh-CN/concepts-messaging/#%E6%B6%88%E6%81%AF)。 -例如,消息的 key,消息发送时间,消息生产时间,用户在消息上自定义的键值对属性等。可以使用 `Message` 接口来获取这些属性。 +如果使用 KeyValue 或者 Struct 类型的Schema, 那么 pulsar `Schema` 讲不会含有类型类信息, 但 `PulsarSchemaTypeInformation` 需要通过传入类型类信息来构造。因此我们提供的API支持用户传入类型类信息。 + +例子如下: + +```java + // Primitive 类型: 不需要提供类型信息 + PulsarDeserializationSchema.pulsarSchema(Schema.INT32); + // Struct 类型 (JSON, Protobuf, Avro, 等等.) + PulsarDeserializationSchema.pulsarSchema(Schema.AVRO(SomeClass), SomeClass.class); + // KeyValue 类型 + PulsarDeserializationSchema.pulsarSchema(Schema.KeyValue(Schema.INT32, Schema.AVRO(SomeClass)), Integer.class, SomeClass.class); +``` -如果用户需要基于这些额外的属性来解析一条消息,可以实现 `PulsarDeserializationSchema` 接口。 -并一定要确保 `PulsarDeserializationSchema.getProducedType()` 方法返回的 `TypeInformation` 是正确的结果。 -Flink 使用 `TypeInformation` 将解析出来的结果序列化传递到下游算子。 +Pulsar 的 `Message` 包含了很多 [额外的属性](https://pulsar.apache.org/docs/zh-CN/concepts-messaging/#%E6%B6%88%E6%81%AF)。例如,消息的 key、消息发送时间、消息生产时间、用户在消息上自定义的键值对属性等。可以使用 `Message` 接口来获取这些属性。 + +如果用户需要基于这些额外的属性来解析一条消息,可以实现 `PulsarDeserializationSchema` 接口。并一定要确保 `PulsarDeserializationSchema.getProducedType()` 方法返回的 `TypeInformation` 是正确的结果。Flink 使用 `TypeInformation` 将解析出来的结果序列化传递到下游算子。 ### Pulsar 订阅 -订阅是命名好的配置规则,指导消息如何投递给消费者。连接器需要提供一个独立的订阅名称,支持 Pulsar 的四种订阅模式: +订阅是命名好的配置规则,指导消息如何投递给消费者。Pulsar Source 需要提供一个独立的订阅名称,支持 Pulsar 的四种订阅模式: - [exclusive(独占)](https://pulsar.apache.org/docs/zh-CN/concepts-messaging/#exclusive) - [shared(共享)](https://pulsar.apache.org/docs/zh-CN/concepts-messaging/#shared%E5%85%B1%E4%BA%AB) - [failover(灾备)](https://pulsar.apache.org/docs/zh-CN/concepts-messaging/#failover%E7%81%BE%E5%A4%87) - [key_shared(key 共享)](https://pulsar.apache.org/docs/zh-CN/concepts-messaging/#key_shared) -当前 Pulsar 连接器里面,`独占` 和 `灾备` 的实现没有区别,如果 Flink 的一个 reader 挂了,连接器会把所有未消费的数据交给其他的 reader 来消费数据。 +当前 Pulsar Source 里,`独占` 和 `灾备` 的实现没有区别,如果 Flink 的一个 reader 挂了,Pulsar Source 会把所有未消费的数据交给其他的 reader 来消费数据。 -默认情况下,如果没有指定订阅类型,连接器使用共享订阅类型(`SubscriptionType.Shared`)。 +默认情况下,如果没有指定订阅类型,Pulsar Source 使用共享订阅类型(`SubscriptionType.Shared`)。 ```java // 名为 "my-shared" 的共享订阅 -PulsarSource.builder().setSubscriptionName("my-shared") +PulsarSource.builder().setSubscriptionName("my-shared"); // 名为 "my-exclusive" 的独占订阅 -PulsarSource.builder().setSubscriptionName("my-exclusive").setSubscriptionType(SubscriptionType.Exclusive) +PulsarSource.builder().setSubscriptionName("my-exclusive").setSubscriptionType(SubscriptionType.Exclusive); ``` -如果想在 Pulsar 连接器里面使用 `key 共享` 订阅,需要提供 `RangeGenerator` 实例。`RangeGenerator` 会生成一组消息 key 的 hash 范围,连接器会基于给定的范围来消费数据。 +如果想在 Pulsar Source 里面使用 `key 共享` 订阅,需要提供 `RangeGenerator` 实例。`RangeGenerator` 会生成一组消息 key 的 hash 范围,Pulsar Source 会基于给定的范围来消费数据。 -Pulsar 连接器也提供了一个名为 `UniformRangeGenerator` 的默认实现,它会基于 flink 数据源的并行度将 hash 范围均分。 +Pulsar Source 也提供了一个名为 `UniformRangeGenerator` 的默认实现,它会基于 flink 数据源的并行度将 hash 范围均分。 ### 起始消费位置 -连接器使用 `setStartCursor(StartCursor)` 方法给定开始消费的位置。内置的消费位置有: +Pulsar Source 使用 `setStartCursor(StartCursor)` 方法给定开始消费的位置。内置的开始消费位置有: -- 从 topic 里面最早的一条消息开始消费。 +- 从 Topic 里面最早的一条消息开始消费。 ```java - StartCursor.earliest() + StartCursor.earliest(); ``` -- 从 topic 里面最新的一条消息开始消费。 +- 从 Topic 里面最新的一条消息开始消费。 ```java - StartCursor.latest() + StartCursor.latest(); ``` - 从给定的消息开始消费。 ```java - StartCursor.fromMessageId(MessageId) + StartCursor.fromMessageId(MessageId); ``` - 与前者不同的是,给定的消息可以跳过,再进行消费。 ```java - StartCursor.fromMessageId(MessageId, boolean) + StartCursor.fromMessageId(MessageId, boolean); ``` -- 从给定的消息时间开始消费。 - ```java - StartCursor.fromMessageTime(long) +- 从给定的消息发布时间开始消费。 + StartCursor.fromPublishTime(long) ``` {{< hint info >}} 每条消息都有一个固定的序列号,这个序列号在 Pulsar 上有序排列,其包含了 ledger、entry、partition 等原始信息,用于在 Pulsar 底层存储上查找到具体的消息。 + Pulsar 称这个序列号为 `MessageId`,用户可以使用 `DefaultImplementation.newMessageId(long ledgerId, long entryId, int partitionIndex)` 创建它。 {{< /hint >}} ### 边界 -Pulsar 连接器同时支持流式和批的消费方式,默认情况下,连接器使用流的方式消费数据。除非任务失败或者被取消,否则连接器将持续消费数据。 -用户可以使用 `setBoundedStopCursor(StopCursor)` 给定停止消费的位置,这种情况下连接器会使用批的方式进行消费。当所有 topic 分区都消费到了停止位置,Flink 任务就会结束。 +Pulsar Source 默认情况下使用流的方式消费数据。除非任务失败或者被取消,否则将持续消费数据。用户可以使用 `setBoundedStopCursor(StopCursor)` 给定停止消费的位置,这种情况下会使用批的方式进行消费。使用流的方式一样可以给定停止位置,使用 `setUnboundedStopCursor(StopCursor)` 方法即可。 -使用流的方式一样可以给定停止位置,使用 `setUnboundedStopCursor(StopCursor)` 方法即可。 +在批模式下,使用 `setBoundedStopCursor(StopCursor)` 来指定一个消费停止位置。 -内置的停止位置如下: +内置的停止消费位置如下: - 永不停止。 ```java - StopCursor.never() + StopCursor.never(); ``` -- 停止于 Pulsar 启动时 topic 里面最新的那条数据。 +- 停止于 Pulsar 启动时 Topic 里面最新的那条数据。 ```java - StopCursor.latest() + StopCursor.latest(); ``` - 停止于某条消息,结果里不包含此消息。 ```java - StopCursor.atMessageId(MessageId) + StopCursor.atMessageId(MessageId); ``` - 停止于某条消息之后,结果里包含此消息。 ```java - StopCursor.afterMessageId(MessageId) + StopCursor.afterMessageId(MessageId); ``` -- 停止于某个给定的消息时间戳。 +- 停止于某个给定的消息发布时间戳,比如 `Message.getPublishTime()`。 ```java - StopCursor.atEventTime(long) + StopCursor.atPublishTime(long); ``` -### 其他配置项 +{{< hint warning >}} +StopCursor.atEventTime(long) 目前已经处于弃用状态。 +{{< /hint >}} + +### Source 配置项 -除了前面提到的配置选项,连接器还提供了丰富的选项供 Pulsar 专家使用,在 builder 类里通过 `setConfig(ConfigOption, T)` 和 `setConfig(Configuration)` 方法给定下述的全部配置。 +除了前面提到的配置选项,Pulsar Source 还提供了丰富的选项供 Pulsar 专家使用,在 builder 类里通过 `setConfig(ConfigOption, T)` 和 `setConfig(Configuration)` 方法给定下述的全部配置。 #### Pulsar Java 客户端配置项 -Pulsar 连接器使用[Java 客户端](https://pulsar.apache.org/docs/en/client-libraries-java/)来创建消费实例,相关的配置定义于 Pulsar 的 `ClientConfigurationData` 内。连接器在 `PulsarOptions` 选项中,定义大部分的可供用户定义的配置。 +Pulsar Source 使用 [Java 客户端](https://pulsar.apache.org/docs/zh-CN/client-libraries-java/)来创建消费实例,相关的配置定义于 Pulsar 的 `ClientConfigurationData` 内。在 `PulsarOptions` 选项中,定义大部分的可供用户定义的配置。 {{< generated/pulsar_client_configuration >}} #### Pulsar 管理 API 配置项 -[管理 API](https://pulsar.apache.org/docs/en/admin-api-overview/) 用于查询 topic 的元数据和用正则订阅的时候的 topic 查找,它与 -Java 客户端共享大部分配置。下面列举的配置只供管理 API 使用,连接器也在 `PulsarOptions` 里予以定义。 +[管理 API](https://pulsar.apache.org/docs/zh-CN/admin-api-overview/) 用于查询 Topic 的元数据和用正则订阅的时候的 Topic 查找,它与 Java 客户端共享大部分配置。下面列举的配置只供管理 API 使用,`PulsarOptions` 包含了这些配置 。 {{< generated/pulsar_admin_configuration >}} #### Pulsar 消费者 API 配置项 -Pulsar 提供了消费者 API 和读者 API 两套 API 来进行数据消费,它们可用于不同的业务场景。 -Flink 上的 Pulsar 连接器使用消费者 API 进行消费,它的配置定义于 Pulsar 的 `ConsumerConfigurationData` 内。连接器将其中大部分的可供用户定义的配置定义于 `PulsarSourceOptions` 内。 +Pulsar 提供了消费者 API 和读者 API 两套 API 来进行数据消费,它们可用于不同的业务场景。Flink 上的 Pulsar Source 使用消费者 API 进行消费,它的配置定义于 Pulsar 的 `ConsumerConfigurationData` 内。Pulsar Source 将其中大部分的可供用户定义的配置定义于 `PulsarSourceOptions` 内。 {{< generated/pulsar_consumer_configuration >}} -#### Pulsar 数据源配置项 +#### Pulsar Source配置项 下述配置主要用于性能调优或者是控制消息确认的行为。如非必要,可以不用强制配置。 @@ -287,8 +290,7 @@ Flink 上的 Pulsar 连接器使用消费者 API 进行消费,它的配置定 ### 动态分区发现 -为了能在启动 Flink 任务之后还能发现在 Pulsar 上扩容的分区或者是新创建的 topic,连接器提供了动态分区发现机制。该机制不需要重启 Flink 任务。 -对选项 `PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS` 设置一个正整数即可启用。 +为了能在启动 Flink 任务之后还能发现在 Pulsar 上扩容的分区或者是新创建的 Topic,Pulsar Source 提供了动态分区发现机制。该机制不需要重启 Flink 任务。对选项 `PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS` 设置一个正整数即可启用。 ```java // 10 秒查询一次分区信息 @@ -302,29 +304,29 @@ PulsarSource.builder() ### 事件时间和水位线 -默认情况下,连接器使用 Pulsar 的 `Message` 里面的时间作为解析结果的时间戳。用户可以使用 `WatermarkStrategy` 来自行解析出想要的消息时间,并向下游传递对应的水位线。 +默认情况下,Pulsar Source 使用 Pulsar 的 `Message` 里面的时间作为解析结果的时间戳。用户可以使用 `WatermarkStrategy` 来自行解析出想要的消息时间,并向下游传递对应的水位线。 ```java -env.fromSource(pulsarSource, new CustomWatermarkStrategy(), "Pulsar Source With Custom Watermark Strategy") +env.fromSource(pulsarSource, new CustomWatermarkStrategy(), "Pulsar Source With Custom Watermark Strategy"); ``` -[这篇文档]({{< ref "docs/dev/datastream/event-time/generating_watermarks.md" >}}) 详细讲解了如何定义 `WatermarkStrategy`。 +[这篇文档]({{< ref "docs/dev/datastream/event-time/generating_watermarks.md" >}})详细讲解了如何定义 `WatermarkStrategy`。 ### 消息确认 -一旦在 topic 上创建了订阅,消息便会[存储](https://pulsar.apache.org/docs/zh-CN/concepts-architecture-overview/#%E6%8C%81%E4%B9%85%E5%8C%96%E5%AD%98%E5%82%A8)在 Pulsar 里。即使没有消费者,消息也不会被丢弃。只有当连接器同 Pulsar 确认此条消息已经被消费,该消息才以某种机制会被移除。连接器支持四种订阅方式,它们的消息确认方式也大不相同。 +一旦在 Topic 上创建了订阅,消息便会[存储](https://pulsar.apache.org/docs/zh-CN/concepts-architecture-overview/#%E6%8C%81%E4%B9%85%E5%8C%96%E5%AD%98%E5%82%A8)在 Pulsar 里。即使没有消费者,消息也不会被丢弃。只有当 Pulsar Source 同 Pulsar 确认此条消息已经被消费,该消息才以某种机制会被移除。Pulsar Source 支持四种订阅方式,它们的消息确认方式也大不相同。 #### 独占和灾备订阅下的消息确认 -`独占` 和 `灾备` 订阅下,连接器使用累进式确认方式。确认某条消息已经被处理时,其前面被消费的消息会自动被置为已读。Pulsar 连接器会在 Flink 完成检查点时将对应时刻消费的消息置为已读,以此来保证 Pulsar 状态与 Flink 状态一致。 +`独占` 和 `灾备` 订阅下,Pulsar Source 使用累进式确认方式。确认某条消息已经被处理时,其前面消息会自动被置为已读。Pulsar Source 会在 Flink 完成检查点时将对应时刻消费的消息置为已读,以此来保证 Pulsar 状态与 Flink 状态一致。 -如果用户没有在 Flink 上启用检查点,连接器可以使用周期性提交来将消费状态提交给 Pulsar,使用配置 `PulsarSourceOptions.PULSAR_AUTO_COMMIT_CURSOR_INTERVAL` 来进行定义。 +如果用户没有在 Flink 上启用检查点,Pulsar Source 可以使用周期性提交来将消费状态提交给 Pulsar,使用配置 `PulsarSourceOptions.PULSAR_AUTO_COMMIT_CURSOR_INTERVAL` 来进行定义。 -需要注意的是,此种场景下,Pulsar 连接器并不依赖于提交到 Pulsar 的状态来做容错。消息确认只是为了能在 Pulsar 端看到对应的消费处理情况。 +需要注意的是,此种场景下,Pulsar Source 并不依赖于提交到 Pulsar 的状态来做容错。消息确认只是为了能在 Pulsar 端看到对应的消费处理情况。 #### 共享和 key 共享订阅下的消息确认 -`共享` 和 `key 共享` 需要依次确认每一条消息,所以连接器在 Pulsar 事务里面进行消息确认,然后将事务提交到 Pulsar。 +`共享` 和 `key 共享` 需要依次确认每一条消息,所以 Pulsar Source 在 Pulsar 事务里面进行消息确认,然后将事务提交到 Pulsar。 首先需要在 Pulsar 的 `borker.conf` 文件里面启用事务: @@ -332,11 +334,331 @@ env.fromSource(pulsarSource, new CustomWatermarkStrategy(), "Pulsar Source With transactionCoordinatorEnabled=true ``` -连接器创建的事务的默认超时时间为 3 小时,请确保这个时间大于 Flink 检查点的间隔。用户可以使用 `PulsarSourceOptions.PULSAR_TRANSACTION_TIMEOUT_MILLIS` 来设置事务的超时时间。 +Pulsar Source 创建的事务的默认超时时间为 3 小时,请确保这个时间大于 Flink 检查点的间隔。用户可以使用 `PulsarSourceOptions.PULSAR_TRANSACTION_TIMEOUT_MILLIS` 来设置事务的超时时间。 + +如果用户无法启用 Pulsar 的事务,或者是因为项目禁用了检查点,需要将 `PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE` 选项设置为 `true`,消息从 Pulsar 消费后会被立刻置为已读。Pulsar Source 无法保证此种场景下的消息一致性。 + +Pulsar Source 在 Pulsar 上使用日志的形式记录某个事务下的消息确认,为了更好的性能,请缩短 Flink 做检查点的间隔。 + +## Pulsar Sink + +Pulsar Sink 连接器可以将经过 Flink 处理后的数据写入一个或多个 Pulsar Topic 或者 Topic 下的某些分区。 + +{{< hint info >}} +Pulsar Sink 基于 Flink 最新的 [Sink API](https://cwiki.apache.org/confluence/display/FLINK/FLIP-191%3A+Extend+unified+Sink+interface+to+support+small+file+compaction) 实现。 + +如果想要使用旧版的使用 `SinkFuntion` 接口实现的 Sink 连接器,可以使用 StreamNative 维护的 [pulsar-flink](https://github.com/streamnative/pulsar-flink)。 +{{< /hint >}} + +### 使用示例 + +Pulsar Sink 使用 builder 类来创建 `PulsarSink` 实例。 + +下面示例展示了如何通过 Pulsar Sink 以“至少一次”的语义将字符串类型的数据发送给 topic1。 + +```java +DataStream stream = ... + +PulsarSink sink = PulsarSink.builder() + .setServiceUrl(serviceUrl) + .setAdminUrl(adminUrl) + .setTopics("topic1") + .setSerializationSchema(PulsarSerializationSchema.flinkSchema(new SimpleStringSchema())) + .setDeliverGuarantee(DeliveryGuarantee.AT_LEAST_ONCE) + .build(); + +stream.sinkTo(sink); +``` + +下列为创建一个 `PulsarSink` 实例必需的属性: + +- Pulsar 数据消费的地址,使用 `setServiceUrl(String)` 方法提供。 +- Pulsar HTTP 管理地址,使用 `setAdminUrl(String)` 方法提供。 +- 需要发送到的 Topic 或者是 Topic 下面的分区,详见[指定写入的topic或者topic分区](#指定写入的topic或者topic分区)。 +- 编码 Pulsar 消息的序列化器,详见[序列化器](#序列化器)。 + +在创建 `PulsarSink` 时,建议使用 `setProducerName(String)` 来指定 `PulsarSink` 内部使用的 Pulsar 生产者名称。这样方便在数据监控页面找到对应的生产者监控指标。 + +### 指定写入的 Topic 或者 Topic 分区 + +`PulsarSink` 指定写入 Topic 的方式和 Pulsar Source [指定消费的 Topic 或者 Topic 分区](#指定消费的-topic-或者-topic-分区)的方式类似。`PulsarSink` 支持以 mixin 风格指定写入的 Topic 或分区。因此,可以指定一组 Topic 或者分区或者是两者都有。 + +```java +// Topic "some-topic1" 和 "some-topic2" +PulsarSink.builder().setTopics("some-topic1", "some-topic2") + +// Topic "topic-a" 的分区 0 和 2 +PulsarSink.builder().setTopics("topic-a-partition-0", "topic-a-partition-2") + +// Topic "topic-a" 以及 Topic "some-topic2" 分区 0 和 2 +PulsarSink.builder().setTopics("topic-a-partition-0", "topic-a-partition-2", "some-topic2") +``` + +动态分区发现默认处于开启状态,这意味着 `PulsarSink` 将会周期性地从 Pulsar 集群中查询 Topic 的元数据来获取可能有的分区数量变更信息。使用 `PulsarSinkOptions.PULSAR_TOPIC_METADATA_REFRESH_INTERVAL` 配置项来指定查询的间隔时间。 + +可以选择实现 `TopicRouter` 接口来自定义[消息路由策略](#消息路由策略)。此外,阅读 [Topic 名称简写](#topic-名称简写)将有助于理解 Pulsar 的分区在 Pulsar 连接器中的配置方式。 + +{{< hint warning >}} +如果在 `PulsarSink` 中同时指定了某个 Topic 和其下属的分区,那么 `PulsarSink` 将会自动将两者合并,仅使用外层的 Topic。 + +举个例子,如果通过 `PulsarSink.builder().setTopics("some-topic1", "some-topic1-partition-0")` 来指定写入的 Topic,那么其结果等价于 `PulsarSink.builder().setTopics("some-topic1")`。 +{{< /hint >}} + +#### 基于消息实例的动态 Topic 指定 + +除了前面说的一开始就指定 Topic 或者是 Topic 分区,你还可以在程序启动后基于消息内容动态指定 Topic,只需要实现 `TopicExtractor` 接口即可。 +`TopicExtractor` 接口还提供了 `TopicMetadataProvider` 用于查询某个 Topic 在 Pulsar 上有多少个分区, +查询结果会缓存并在 `PulsarSinkOptions.PULSAR_TOPIC_METADATA_REFRESH_INTERVAL` 之后失效。 + +`TopicExtractor` 的返回结果支持带分区信息和不带分区信息的 Topic。 + +1. 当返回结果里没有分区信息时,我们会查询对应的分区大小,生成所有的分区 Topic,然后传递给 `TopicRouter` 用于路由。分区信息将会被缓存 `PulsarSinkOptions.PULSAR_TOPIC_METADATA_REFRESH_INTERVAL`。 +2. 如果你的返回结果里面提供了分区信息,我们则会什么都不做,直接传递给下游。 + +### 序列化器 + +序列化器(`PulsarSerializationSchema`)负责将 Flink 中的每条记录序列化成 byte 数组,并通过网络发送至指定的写入 Topic。和 Pulsar Source 类似的是,序列化器同时支持使用基于 Flink 的 `SerializationSchema` 接口实现序列化器和使用 Pulsar 原生的 `Schema` 类型实现的序列化器。不过序列化器并不支持 Pulsar 的 `Schema.AUTO_PRODUCE_BYTES()`。 + +如果不需要指定 [Message](https://pulsar.apache.org/api/client/2.9.0-SNAPSHOT/org/apache/pulsar/client/api/Message.html) 接口中提供的 key 或者其他的消息属性,可以从上述 2 种预定义的 `PulsarSerializationSchema` 实现中选择适合需求的一种使用。 + +- 使用 Pulsar 的 [Schema](https://pulsar.apache.org/docs/zh-CN/schema-understand/) 来序列化 Flink 中的数据。 + ```java + // 原始数据类型 + PulsarSerializationSchema.pulsarSchema(Schema) + + // 有结构数据类型(JSON、Protobuf、Avro 等) + PulsarSerializationSchema.pulsarSchema(Schema, Class) + + // 键值对类型 + PulsarSerializationSchema.pulsarSchema(Schema, Class, Class) + ``` +- 使用 Flink 的 `SerializationSchema` 来序列化数据。 + ```java + PulsarSerializationSchema.flinkSchema(SerializationSchema) + ``` + +同时使用 `PulsarSerializationSchema.pulsarSchema()` 以及在 builder 中指定 `PulsarSinkBuilder.enableSchemaEvolution()` 可以启用 [Schema evolution](https://pulsar.apache.org/docs/zh-CN/schema-evolution-compatibility/#schema-evolution) 特性。该特性会使用 Pulsar Broker 端提供的 Schema 版本兼容性检测以及 Schema 版本演进。下列示例展示了如何启用 Schema Evolution。 + +```java +Schema schema = Schema.AVRO(SomePojo.class); +PulsarSerializationSchema pulsarSchema = PulsarSerializationSchema.pulsarSchema(schema, SomePojo.class); + +PulsarSink sink = PulsarSink.builder() + ... + .setSerializationSchema(pulsarSchema) + .enableSchemaEvolution() + .build(); +``` + +{{< hint warning >}} +如果想要使用 Pulsar 原生的 Schema 序列化消息而不需要 Schema Evolution 特性,那么写入的 Topic 会使用 `Schema.BYTES` 作为消息的 Schema,对应 Topic 的消费者需要自己负责反序列化的工作。 + +例如,如果使用 `PulsarSerializationSchema.pulsarSchema(Schema.STRING)` 而不使用 `PulsarSinkBuilder.enableSchemaEvolution()`。那么在写入 Topic 中所记录的消息 Schema 将会是 `Schema.BYTES`。 +{{< /hint >}} + +### 消息路由策略 + +在 Pulsar Sink 中,消息路由发生在于分区之间,而非上层 Topic。对于给定 Topic 的情况,路由算法会首先会查询出 Topic 之上所有的分区信息,并在这些分区上实现消息的路由。Pulsar Sink 默认提供 2 种路由策略的实现。 + +- `KeyHashTopicRouter`:使用消息的 key 对应的哈希值来取模计算出消息对应的 Topic 分区。 + + 使用此路由可以将具有相同 key 的消息发送至同一个 Topic 分区。消息的 key 可以在自定义 `PulsarSerializationSchema` 时,在 `serialize()` 方法内使用 `PulsarMessageBuilder.key(String key)` 来予以指定。 + + 如果消息没有包含 key,此路由策略将从 Topic 分区中随机选择一个发送。 + + 可以使用 `MessageKeyHash.JAVA_HASH` 或者 `MessageKeyHash.MURMUR3_32_HASH` 两种不同的哈希算法来计算消息 key 的哈希值。使用 `PulsarSinkOptions.PULSAR_MESSAGE_KEY_HASH` 配置项来指定想要的哈希算法。 + +- `RoundRobinRouter`:轮换使用用户给定的 Topic 分区。 + + 消息将会轮替地选取 Topic 分区,当往某个 Topic 分区里写入指定数量的消息后,将会轮换至下一个 Topic 分区。使用 `PulsarSinkOptions.PULSAR_BATCHING_MAX_MESSAGES` 指定向一个 Topic 分区中写入的消息数量。 + +还可以通过实现 `TopicRouter` 接口来自定义消息路由策略,请注意 TopicRouter 的实现需要能被序列化。 + +在 `TopicRouter` 内可以指定任意的 Topic 分区(即使这个 Topic 分区不在 `setTopics()` 指定的列表中)。因此,当使用自定义的 `TopicRouter` 时,`PulsarSinkBuilder.setTopics` 选项是可选的。 + +```java +@PublicEvolving +public interface TopicRouter extends Serializable { + + String route(IN in, List partitions, PulsarSinkContext context); + + default void open(SinkConfiguration sinkConfiguration) { + // 默认无操作 + } +} +``` + +{{< hint info >}} +如前文所述,Pulsar 分区的内部被实现为一个无分区的 Topic,一般情况下 Pulsar 客户端会隐藏这个实现,并且提供内置的消息路由策略。Pulsar Sink 并没有使用 Pulsar 客户端提供的路由策略和封装,而是使用了 Pulsar 客户端更底层的 API 自行实现了消息路由逻辑。这样做的主要目的是能够在属于不同 Topic 的分区之间定义更灵活的消息路由策略。 + +详情请参考 Pulsar 的 [partitioned topics](https://pulsar.apache.org/docs/zh-CN/cookbooks-partitioned/)。 +{{< /hint >}} + +### 发送一致性 + +`PulsarSink` 支持三种发送一致性。 + +- `NONE`:Flink 应用运行时可能出现数据丢失的情况。在这种模式下,Pulsar Sink 发送消息后并不会检查消息是否发送成功。此模式具有最高的吞吐量,可用于一致性没有要求的场景。 +- `AT_LEAST_ONCE`:每条消息**至少有**一条对应消息发送至 Pulsar,发送至 Pulsar 的消息可能会因为 Flink 应用重启而出现重复。 +- `EXACTLY_ONCE`:每条消息**有且仅有**一条对应消息发送至 Pulsar。发送至 Pulsar 的消息不会有重复也不会丢失。Pulsar Sink 内部依赖 [Pulsar 事务](https://pulsar.apache.org/docs/zh-CN/transactions/)和两阶段提交协议来保证每条记录都能正确发往 Pulsar。 + +### 消息延时发送 -如果用户无法启用 Pulsar 的事务,或者是因为项目禁用了检查点,需要将 `PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE` 选项设置为 `true`,消息从 Pulsar 消费后会被立刻置为已读。连接器无法保证此种场景下的消息一致性。 +[消息延时发送](https://pulsar.apache.org/docs/zh-CN/next/concepts-messaging/#%E6%B6%88%E6%81%AF%E5%BB%B6%E8%BF%9F%E4%BC%A0%E9%80%92)特性可以让指定发送的每一条消息需要延时一段时间后才能被下游的消费者所消费。当延时消息发送特性启用时,Pulsar Sink 会**立刻**将消息发送至 Pulsar Broker。但该消息在指定的延迟时间到达前将会保持对下游消费者不可见。 -连接器在 Pulsar 上使用日志的形式记录某个事务下的消息确认,为了更好的性能,请缩短 Flink 做检查点的间隔。 +消息延时发送仅在 `Shared` 订阅模式下有效,在 `Exclusive` 和 `Failover` 模式下该特性无效。 + +可以使用 `MessageDelayer.fixed(Duration)` 创建一个 `MessageDelayer` 来为所有消息指定恒定的接收时延,或者实现 `MessageDelayer` 接口来为不同的消息指定不同的接收时延。 + +{{< hint warning >}} +消息对下游消费者的可见时间应当基于 `PulsarSinkContext.processTime() `计算得到。 +{{< /hint >}} + +### Sink 配置项 + +可以在 builder 类里通过 `setConfig(ConfigOption, T)` 和 `setConfig(Configuration)` 方法给定下述的全部配置。 + +#### PulsarClient 和 PulsarAdmin 配置项 + +Pulsar Sink 和 Pulsar Source 公用的配置选项可参考 + +- [Pulsar Java 客户端配置项](#pulsar-java-客户端配置项) +- [Pulsar 管理 API 配置项](#pulsar-管理-API-配置项) + +#### Pulsar 生产者 API 配置项 + +Pulsar Sink 使用生产者 API 来发送消息。Pulsar 的 `ProducerConfigurationData` 中大部分的配置项被映射为 `PulsarSinkOptions` 里的选项。 + +{{< generated/pulsar_producer_configuration >}} + +#### Pulsar Sink 配置项 + +下述配置主要用于性能调优或者是控制消息确认的行为。如非必要,可以不用考虑配置。 + +{{< generated/pulsar_sink_configuration >}} + +### Sink 监控指标 + +下列表格列出了当前 Sink 支持的监控指标,前 6 个指标是 [FLIP-33: Standardize Connector Metrics]([https://cwiki.apache.org/confluence/display/FLINK/FLIP-33%3A+Standardize+Connector+Metrics](https://cwiki.apache.org/confluence/display/FLINK/FLIP-33%3A+Standardize+Connector+Metrics)) 中规定的 Sink 连接器应当支持的标准指标。 + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
ScopeMetricsUser VariablesDescriptionType
OperatornumBytesOutn/aPulsar Sink 启动后总共发出的字节数Counter
numBytesOutPerSecondn/a每秒发送的字节数Meter
numRecordsOutn/aPulsar Sink 启动后总共发出的消息数Counter
numRecordsOutPerSecondn/a每秒发送的消息数Meter
numRecordsOutErrorsn/a总共发送消息失败的次数Counter
currentSendTimen/a最近一条消息从被放入客户端缓冲队列到收到消息确认的时间Gauge
PulsarSink.numAcksReceivedn/a总共收到的确认数Counter
PulsarSink.sendLatencyMaxn/a所有生产者的最大发送延迟Gauge
PulsarSink.producer."ProducerName".sendLatency50PctProducerName某个生产者在过去的一个窗口内的发送延迟的中位数Gauge
PulsarSink.producer."ProducerName".sendLatency75PctProducerName某个生产者在过去的一个窗口内的发送延迟的 75 百分位数Gauge
PulsarSink.producer."ProducerName".sendLatency95PctProducerName某个生产者在过去的一个窗口内的发送延迟的 95 百分位数Gauge
PulsarSink.producer."ProducerName".sendLatency99PctProducerName某个生产者在过去的一个窗口内的发送延迟的 99 百分位数Gauge
PulsarSink.producer."ProducerName".sendLatency999PctProducerName某个生产者在过去的一个窗口内的发送延迟的 99.9 百分位数Gauge
+ +{{< hint info >}} +指标 `numBytesOut`、`numRecordsOut` 和 `numRecordsOutErrors` 从 Pulsar Producer 实例的监控指标中获得。 + +`currentSendTime` 记录了最近一条消息从放入生产者的缓冲队列到消息被消费确认所耗费的时间。这项指标在 `NONE` 发送一致性下不可用。 +{{< /hint >}} + +默认情况下,Pulsar 生产者每隔 60 秒才会刷新一次监控数据,然而 Pulsar Sink 每 500 毫秒就会从 Pulsar 生产者中获得最新的监控数据。因此 `numRecordsOut`、`numBytesOut`、`numAcksReceived` 以及 `numRecordsOutErrors` 4 个指标实际上每 60 秒才会刷新一次。 + +如果想要更高地刷新评率,可以通过 `builder.setConfig(PulsarOptions.PULSAR_STATS_INTERVAL_SECONDS. 1L)` 来将 Pulsar 生产者的监控数据刷新频率调整至相应值(最低为1s)。 + +`numBytesOutRate` 和 `numRecordsOutRate` 指标是 Flink 内部通过 `numBytesOut` 和 `numRecordsOut` 计数器,在一个 60 秒的窗口内计算得到的。 + +### 设计思想简述 + +Pulsar Sink 遵循 [FLIP-191](https://cwiki.apache.org/confluence/display/FLINK/FLIP-191%3A+Extend+unified+Sink+interface+to+support+small+file+compaction) 中定义的 Sink API 设计。 + +#### 无状态的 SinkWriter + +在 `EXACTLY_ONCE` 一致性下,Pulsar Sink 不会将事务相关的信息存放于检查点快照中。这意味着当 Flink 应用重启时,Pulsar Sink 会创建新的事务实例。上一次运行过程中任何未提交事务中的消息会因为超时中止而无法被下游的消费者所消费。这样的设计保证了 SinkWriter 是无状态的。 + +#### Pulsar Schema Evolution + +[Pulsar Schema Evolution](https://pulsar.apache.org/docs/zh-CN/schema-evolution-compatibility/) 允许用户在一个 Flink 应用程序中使用的数据模型发生特定改变后(比如向基于 ARVO 的 POJO 类中增加或删除一个字段),仍能使用同一个 Flink 应用程序的代码。 + +可以在 Pulsar 集群内指定哪些类型的数据模型的改变是被允许的,详情请参阅 [Pulsar Schema Evolution](https://pulsar.apache.org/docs/zh-CN/schema-evolution-compatibility/)。 ## 升级至最新的连接器 @@ -347,6 +669,11 @@ transactionCoordinatorEnabled=true ## 问题诊断 -使用 Flink 和 Pulsar 交互时如果遇到问题,一定要牢记 Flink 只使用了 Pulsar 的[Java 客户端](https://pulsar.apache.org/docs/zh-CN/client-libraries-java/) 和[管理 API](https://pulsar.apache.org/docs/zh-CN/admin-api-overview/)。用户遇到的问题很有可能与 Flink 无关,请先升级 Pulsar 的版本、Pulsar 客户端的版本、或者修改 Pulsar 的配置,Pulsar 连接器的配置来尝试解决问题。 +使用 Flink 和 Pulsar 交互时如果遇到问题,由于 Flink 内部实现只是基于 Pulsar 的 [Java 客户端](https://pulsar.apache.org/docs/zh-CN/client-libraries-java/)和[管理 API](https://pulsar.apache.org/docs/zh-CN/admin-api-overview/) 而开发的。 + +用户遇到的问题可能与 Flink 无关,请先升级 Pulsar 的版本、Pulsar 客户端的版本,或者修改 Pulsar 的配置、Pulsar 连接器的配置来尝试解决问题。 + +### 在 Java 11 上使用不稳定 +Pulsar connector 在 Java 11 中有一些尚未修复的问题。我们当前推荐在 Java 8 环境中运行Pulsar connector. {{< top >}} diff --git a/docs/content.zh/docs/connectors/datastream/rabbitmq.md b/docs/content.zh/docs/connectors/datastream/rabbitmq.md index 7c3a2a10c452f..668987fe6e9f7 100644 --- a/docs/content.zh/docs/connectors/datastream/rabbitmq.md +++ b/docs/content.zh/docs/connectors/datastream/rabbitmq.md @@ -40,7 +40,7 @@ Flink 自身既没有复用 "RabbitMQ AMQP Java Client" 的代码,也没有将 {{< artifact flink-connector-rabbitmq >}} -注意连接器现在没有包含在二进制发行版中。集群执行的相关信息请参考 [这里]({{< ref "docs/dev/datastream/project-configuration" >}}). +注意连接器现在没有包含在二进制发行版中。集群执行的相关信息请参考 [这里]({{< ref "docs/dev/configuration/overview" >}}). ### 安装 RabbitMQ 安装 RabbitMQ 请参考 [RabbitMQ 下载页面](http://www.rabbitmq.com/download.html)。安装完成之后,服务会自动拉起,应用程序就可以尝试连接到 RabbitMQ 了。 diff --git a/docs/content.zh/docs/connectors/table/elasticsearch.md b/docs/content.zh/docs/connectors/table/elasticsearch.md index 339d9d56f82fa..3395eda13703f 100644 --- a/docs/content.zh/docs/connectors/table/elasticsearch.md +++ b/docs/content.zh/docs/connectors/table/elasticsearch.md @@ -40,6 +40,8 @@ Elasticsearch 连接器允许将数据写入到 Elasticsearch 引擎的索引中 {{< sql_download_table "elastic" >}} +Elasticsearch 连接器不是二进制发行版的一部分,请查阅[这里]({{< ref "docs/dev/configuration/overview" >}})了解如何在集群运行中引用 Elasticsearch 连接器。 + 如何创建 Elasticsearch 表 ---------------- @@ -128,11 +130,27 @@ CREATE TABLE myUserTable ( 用于连接 Elasticsearch 实例的密码。如果配置了username,则此选项也必须配置为非空字符串。 -
sink.delivery-guarantee
- 可选 - NONE +
failure-handler
+ optional + fail String - 提交时可选的传输保障策略。有效值为 NONE 或者 AT_LEAST_ONCE。 + 对 Elasticsearch 请求失败情况下的失败处理策略。有效策略为: +
    +
  • fail:如果请求失败并因此导致作业失败,则抛出异常。
  • +
  • ignore:忽略失败并放弃请求。
  • +
  • retry-rejected:重新添加由于队列容量饱和而失败的请求。
  • +
  • 自定义类名称:使用 ActionRequestFailureHandler 的子类进行失败处理。
  • +
+ + + +
sink.flush-on-checkpoint
+ optional + true + Boolean + 在进行 checkpoint 时是否保证刷出缓冲区中的数据。如果关闭这一选项,在进行checkpoint时 sink 将不再为所有进行 + 中的请求等待 Elasticsearch 的执行完成确认。因此,在这种情况下 sink 将不对至少一次的请求的一致性提供任何保证。 +
sink.bulk-flush.max-actions
@@ -164,11 +182,11 @@ CREATE TABLE myUserTable (
sink.bulk-flush.backoff.strategy
可选 - NONE + DISABLED String 指定在由于临时请求错误导致任何 flush 操作失败时如何执行重试。有效策略为:
    -
  • NONE:不执行重试,即第一次请求错误后失败。
  • +
  • DISABLED:不执行重试,即第一次请求错误后失败。
  • CONSTANT:等待重试之间的回退延迟。
  • EXPONENTIAL:先等待回退延迟,然后在重试之间指数递增。
@@ -188,13 +206,6 @@ CREATE TABLE myUserTable ( Duration 每次退避尝试之间的延迟。对于 CONSTANT 退避策略,该值是每次重试之间的延迟。对于 EXPONENTIAL 退避策略,该值是初始的延迟。 - -
sink.parallelism
- 可选 - (none) - Integer - 定义 Elasticsearch sink 算子的并行度。默认情况下,并行度由框架定义为与上游串联的算子相同。 -
connection.path-prefix
可选 diff --git a/docs/content.zh/docs/connectors/table/hbase.md b/docs/content.zh/docs/connectors/table/hbase.md index 85b376f722673..2ff3fa49ea176 100644 --- a/docs/content.zh/docs/connectors/table/hbase.md +++ b/docs/content.zh/docs/connectors/table/hbase.md @@ -40,6 +40,7 @@ HBase 连接器在 upsert 模式下运行,可以使用 DDL 中定义的主键 {{< sql_download_table "hbase" >}} +HBase 连接器不是二进制发行版的一部分,请查阅[这里]({{< ref "docs/dev/configuration/overview" >}})了解如何在集群运行中引用 HBase 连接器。 如何使用 HBase 表 ---------------- 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 6cc9e3041b240..dc2e461fd7f55 100644 --- a/docs/content.zh/docs/connectors/table/hive/hive_catalog.md +++ b/docs/content.zh/docs/connectors/table/hive/hive_catalog.md @@ -137,28 +137,23 @@ Time taken: 0.028 seconds, Fetched: 0 row(s) ``` -#### step 2: configure Flink cluster and SQL CLI - -Add all Hive dependencies to `/lib` dir in Flink distribution, and modify SQL CLI's yaml config file `sql-cli-defaults.yaml` as following: - -```yaml - -execution: - type: streaming - ... - current-catalog: myhive # set the HiveCatalog as the current catalog of the session - current-database: mydatabase - -catalogs: - - name: myhive - type: hive - hive-conf-dir: /opt/hive-conf # contains hive-site.xml +#### step 2: start SQL Client, and create a Hive catalog with Flink SQL DDL + +Add all Hive dependencies to `/lib` dir in Flink distribution, and create a Hive catalog in Flink SQL CLI as following: + +```bash + +Flink SQL> CREATE CATALOG myhive WITH ( + 'type' = 'hive', + 'hive-conf-dir' = '/opt/hive-conf' +); + ``` #### step 3: set up a Kafka cluster -Bootstrap a local Kafka 2.3.0 cluster with a topic named "test", and produce some simple data to the topic as tuple of name and age. +Bootstrap a local Kafka cluster with a topic named "test", and produce some simple data to the topic as tuple of name and age. ```bash @@ -180,11 +175,12 @@ john,21 ``` -#### step 4: start SQL Client, and create a Kafka table with Flink SQL DDL +#### step 4: create a Kafka table with Flink SQL DDL -Start Flink SQL Client, create a simple Kafka 2.3.0 table via DDL, and verify its schema. +Create a simple Kafka table with Flink SQL DDL, and verify its schema. ```bash +Flink SQL> USE CATALOG myhive; Flink SQL> CREATE TABLE mykafka (name String, age Int) WITH ( 'connector.type' = 'kafka', 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 3101f0db990aa..d591457ce70ec 100644 --- a/docs/content.zh/docs/connectors/table/hive/hive_dialect.md +++ b/docs/content.zh/docs/connectors/table/hive/hive_dialect.md @@ -416,3 +416,6 @@ Flink SQL> select * from tbl cluster by key; -- run cluster by - 虽然所有 Hive 版本支持相同的语法,但是一些特定的功能是否可用仍取决于你使用的[Hive 版本]({{< ref "docs/connectors/table/hive/overview" >}}#支持的hive版本)。例如,更新数据库位置 只在 Hive-2.4.0 或更高版本支持。 - 执行 DML 和 DQL 时应该使用 [HiveModule]({{< ref "docs/connectors/table/hive/hive_functions" >}}#use-hive-built-in-functions-via-hivemodule) 。 +- 从 Flink 1.15版本开始,在使用 Hive 方言抛出以下异常时,请尝试用 opt 目录下的 flink-table-planner_2.12 jar 包来替换 lib 目录下的 flink-table-planner-loader jar 包。具体原因请参考 [FLINK-25128](https://issues.apache.org/jira/browse/FLINK-25128)。 + {{error}} + diff --git a/docs/content.zh/docs/connectors/table/jdbc.md b/docs/content.zh/docs/connectors/table/jdbc.md index ec8adaa522113..9a812fc891815 100644 --- a/docs/content.zh/docs/connectors/table/jdbc.md +++ b/docs/content.zh/docs/connectors/table/jdbc.md @@ -44,6 +44,8 @@ JDBC 连接器允许使用 JDBC 驱动向任意类型的关系型数据库读取 {{< sql_download_table "jdbc" >}} +JDBC 连接器不是二进制发行版的一部分,请查阅[这里]({{< ref "docs/dev/configuration/overview" >}})了解如何在集群运行中引用 JDBC 连接器。 + 在连接到具体数据库时,也需要对应的驱动依赖,目前支持的驱动如下: | Driver | Group Id | Artifact Id | JAR | @@ -53,7 +55,7 @@ JDBC 连接器允许使用 JDBC 驱动向任意类型的关系型数据库读取 | PostgreSQL | `org.postgresql` | `postgresql` | [下载](https://jdbc.postgresql.org/download.html) | | Derby | `org.apache.derby` | `derby` | [下载](http://db.apache.org/derby/derby_downloads.html) | | -当前,JDBC 连接器和驱动不在 Flink 二进制发布包中,请参阅[这里]({{< ref "docs/dev/datastream/project-configuration" >}})了解在集群上执行时何连接它们。 +当前,JDBC 连接器和驱动不在 Flink 二进制发布包中,请参阅[这里]({{< ref "docs/dev/configuration" >}})了解在集群上执行时何连接它们。 diff --git a/docs/content.zh/docs/connectors/table/kafka.md b/docs/content.zh/docs/connectors/table/kafka.md index 6e3d3f5873848..d02d1cfba2b39 100644 --- a/docs/content.zh/docs/connectors/table/kafka.md +++ b/docs/content.zh/docs/connectors/table/kafka.md @@ -36,7 +36,7 @@ Kafka 连接器提供从 Kafka topic 中消费和写入数据的能力。 {{< sql_download_table "kafka" >}} -Kafka 连接器目前并不包含在 Flink 的二进制发行版中,请查阅 [这里]({{< ref "docs/dev/datastream/project-configuration" >}}) 了解如何在集群运行中引用 Kafka 连接器。 +Kafka 连接器目前并不包含在 Flink 的二进制发行版中,请查阅[这里]({{< ref "docs/dev/configuration/overview" >}})了解如何在集群运行中引用 Kafka 连接器。 如何创建 Kafka 表 ---------------- diff --git a/docs/content.zh/docs/connectors/table/kinesis.md b/docs/content.zh/docs/connectors/table/kinesis.md index 93d765719ca17..e706c32d2d5ef 100644 --- a/docs/content.zh/docs/connectors/table/kinesis.md +++ b/docs/content.zh/docs/connectors/table/kinesis.md @@ -36,6 +36,8 @@ Dependencies {{< sql_download_table "kinesis" >}} +Kinesis 连接器目前并不包含在 Flink 的二进制发行版中,请查阅[这里]({{< ref "docs/dev/configuration/overview" >}})了解如何在集群运行中引用 Kinesis 连接器。 + How to create a Kinesis data stream table ----------------------------------------- diff --git a/docs/content.zh/docs/connectors/table/overview.md b/docs/content.zh/docs/connectors/table/overview.md index 01fa0f849fac9..03fb840f5e6e4 100644 --- a/docs/content.zh/docs/connectors/table/overview.md +++ b/docs/content.zh/docs/connectors/table/overview.md @@ -95,6 +95,8 @@ Flink natively support various connectors. The following tables list all availab {{< top >}} +请查阅[配置]({{< ref "docs/dev/configuration/connector" >}})小节了解如何添加连接器依赖。 + How to use connectors -------- diff --git a/docs/content.zh/docs/connectors/table/upsert-kafka.md b/docs/content.zh/docs/connectors/table/upsert-kafka.md index 298a5f97aaecc..40df1fa203f27 100644 --- a/docs/content.zh/docs/connectors/table/upsert-kafka.md +++ b/docs/content.zh/docs/connectors/table/upsert-kafka.md @@ -40,6 +40,8 @@ Upsert Kafka 连接器支持以 upsert 方式从 Kafka topic 中读取数据并 {{< sql_download_table "upsert-kafka" >}} +Upsert Kafka 连接器不是二进制发行版的一部分,请查阅[这里]({{< ref "docs/dev/configuration/overview" >}})了解如何在集群运行中引用 Upsert Kafka 连接器。 + 完整示例 ---------------- diff --git a/docs/content.zh/docs/deployment/cli.md b/docs/content.zh/docs/deployment/cli.md index 68ec237f4f1d4..c0af1a426a27f 100644 --- a/docs/content.zh/docs/deployment/cli.md +++ b/docs/content.zh/docs/deployment/cli.md @@ -418,6 +418,22 @@ $ ./bin/flink run \ --python examples/python/table/word_count.py ``` +- Run a PyFlink job using a [YARN cluster in Application Mode]({{< ref "docs/deployment/resource-providers/yarn" >}}#application-mode): +```bash +$ ./bin/flink run-application -t yarn-application \ + -Djobmanager.memory.process.size=1024m \ + -Dtaskmanager.memory.process.size=1024m \ + -Dyarn.application.name= \ + -Dyarn.ship-files=/path/to/shipfiles \ + -pyarch shipfiles/venv.zip \ + -pyclientexec venv.zip/venv/bin/python3 \ + -pyexec venv.zip/venv/bin/python3 \ + -py shipfiles/word_count.py +``` +Note It assumes that the Python dependencies needed to execute the job are already placed in the directory `/path/to/shipfiles`. For example, it should contain venv.zip and word_count.py for the above example. + +Note As it executes the job on the JobManager in YARN application mode, the paths specified in `-pyarch` and `-py` are paths relative to `shipfiles` which is the directory name of the shipped files. + - Run a PyFlink application on a native Kubernetes cluster having the cluster ID ``, it requires a docker image with PyFlink installed, please refer to [Enabling PyFlink in docker]({{< ref "docs/deployment/resource-providers/standalone/docker" >}}#enabling-python): ```bash $ ./bin/flink run-application \ diff --git a/docs/content.zh/docs/deployment/config.md b/docs/content.zh/docs/deployment/config.md index 9348ecc2518f4..912ac8bb1d859 100644 --- a/docs/content.zh/docs/deployment/config.md +++ b/docs/content.zh/docs/deployment/config.md @@ -166,6 +166,10 @@ The JobManager ensures consistency during recovery across TaskManagers. For the {{< generated/common_high_availability_section >}} +**Options for the JobResultStore in high-availability setups** + +{{< generated/common_high_availability_jrs_section >}} + **Options for high-availability setups with ZooKeeper** {{< generated/common_high_availability_zk_section >}} diff --git a/docs/content.zh/docs/deployment/elastic_scaling.md b/docs/content.zh/docs/deployment/elastic_scaling.md index c1effbdf9153f..b44c538a58287 100644 --- a/docs/content.zh/docs/deployment/elastic_scaling.md +++ b/docs/content.zh/docs/deployment/elastic_scaling.md @@ -149,5 +149,47 @@ Adaptive 调度器可以通过[所有在名字包含 `adaptive-scheduler` 的配 - **空闲 Slot**: 如果 Slot 共享组的最大并行度不相等,提供给 Adaptive 调度器所使用的的 Slot 可能不会被使用。 - 扩缩容事件会触发 Job 和 Task 重启,Task 重试的次数也会增加。 +## Adaptive Batch Scheduler + +Adaptive Batch Scheduler 是一种可以自动推导每个算子并行度的批作业调度器。如果算子未设置并行度,调度器将根据其消费的数据量的大小来推导其并行度。这可以带来诸多好处: +- 批作业用户可以从并行度调优中解脱出来 +- 根据数据量自动推导并行度可以更好地适应每天变化的数据量 +- SQL作业中的算子也可以分配不同的并行度 + +### 用法 + +使用 Adaptive Batch Scheduler 自动推导算子的并行度,需要: +- 启用 Adaptive Batch Scheduler +- 配置算子的并行度为 `-1` + +#### 启用 Adaptive Batch Scheduler +为了启用 Adaptive Batch Scheduler, 你需要: +- 配置 `jobmanager.scheduler: AdaptiveBatch` +- 由于 ["只支持所有数据交换都为 BLOCKING 模式的作业"](#局限性-2), 需要将 [`execution.batch-shuffle-mode`]({{< ref "docs/deployment/config" >}}#execution-batch-shuffle-mode) 配置为 `ALL-EXCHANGES-BLOCKING`(默认值) 。 + +除此之外,使用 Adaptive Batch Scheduler 时,以下相关配置也可以调整: +- [`jobmanager.adaptive-batch-scheduler.min-parallelism`]({{< ref "docs/deployment/config" >}}#jobmanager-adaptive-batch-scheduler-min-parallelism): 允许自动设置的并行度最小值。需要配置为 2 的幂,否则也会被自动调整为最接近且大于其的 2 的幂。 +- [`jobmanager.adaptive-batch-scheduler.max-parallelism`]({{< ref "docs/deployment/config" >}}#jobmanager-adaptive-batch-scheduler-max-parallelism): 允许自动设置的并行度最大值。需要配置为 2 的幂,否则也会被自动调整为最接近且小于其的 2 的幂。 +- [`jobmanager.adaptive-batch-scheduler.avg-data-volume-per-task`]({{< ref "docs/deployment/config" >}}#jobmanager-adaptive-batch-scheduler-avg-data-volume-per-task): 期望每个任务平均处理的数据量大小。由于顶点的并行度会被调整为 2^N,因此实际每个任务平均处理的数据量大小将是该值的 0.75~1.5 倍。 另外需要注意的是,当出现数据倾斜,或者确定的并行度达到最大并行度(由于数据过多)时,一些任务实际处理的数据可能会远远超过这个值。 +- [`jobmanager.adaptive-batch-scheduler.default-source-parallelism`]({{< ref "docs/deployment/config" >}}#jobmanager-adaptive-batch-scheduler-default-source-parallelism): source 算子的默认并行度 + +#### 配置算子的并行度为 `-1` +Adaptive Batch Scheduler 只会为用户未指定并行度的算子(并行度为 `-1`)推导并行度。 所以如果你想自动推导算子的并行度,需要进行以下配置: +- 配置 `parallelism.default: -1` +- 对于 SQL 作业,需要配置 `table.exec.resource.default-parallelism: -1` +- 对于 DataStream/DataSet 作业,不要在作业中通过算子的 `setParallelism()` 方法来指定并行度 +- 对于 DataStream/DataSet 作业,不要在作业中通过 `StreamExecutionEnvironment/ExecutionEnvironment` 的 `setParallelism()` 方法来指定并行度 + +### 性能调优 + +1. 建议使用 [Sort Shuffle](https://flink.apache.org/2021/10/26/sort-shuffle-part1.html) 并且设置 [`taskmanager.network.memory.buffers-per-channel`]({{< ref "docs/deployment/config" >}}#taskmanager-network-memory-buffers-per-channel) 为 `0`。 这会解耦并行度与需要的网络内存,对于大规模作业,这样可以降低遇到 "Insufficient number of network buffers" 错误的可能性。 +2. 建议将 [`jobmanager.adaptive-batch-scheduler.max-parallelism`]({{< ref "docs/deployment/config" >}}#jobmanager-adaptive-batch-scheduler-max-parallelism) 设置为最坏情况下预期需要的并行度。不建议配置太大的值,否则可能会影响性能。这个配置项会影响上游任务产出的 subpartition 的数量,过多的 subpartition 可能会影响 hash shuffle 的性能,或者由于小包影响网络传输的性能。 + +### 局限性 +- **只支持批作业**: Adaptive Batch Scheduler 只支持批作业。当提交的是一个流作业时,会抛出异常。 +- **只支持所有数据交换都为 BLOCKING 模式的作业**: 目前 Adaptive Batch Scheduler 只支持 [shuffle mode]({{< ref "docs/deployment/config" >}}#execution-batch-shuffle-mode) 为 ALL-EXCHANGES-BLOCKING 的作业。 +- **推导出的并行度是 2 的幂**: 为了使子分区可以均匀分配给下游任务,[`jobmanager.adaptive-batch-scheduler.max-parallelism`]({{< ref "docs/deployment/config" >}}#jobmanager-adaptive-batch-scheduler-max-parallelism) 应该被配置为 2^N, 推导出的并行度会是 2^M, 且满足 M <= N。 +- **不支持 FileInputFormat 类型的 source**: 不支持 FileInputFormat 类型的 source, 包括 `StreamExecutionEnvironment#readFile(...)` `StreamExecutionEnvironment#readTextFile(...)` 和 `StreamExecutionEnvironment#createInput(FileInputFormat, ...)`。 当使用 Adaptive Batch Scheduler 时,用户应该使用新版的 Source API ([FileSystem DataStream Connector]({{< ref "docs/connectors/datastream/filesystem.md" >}}) 或 [FileSystem SQL Connector]({{< ref "docs/connectors/table/filesystem.md" >}})) 来读取文件. +- **Web UI 上展示的上游输出的数据量和下游收到的数据量可能不一致**: 在使用 Adaptive Batch Scheduler 时,对于 broadcast 边,上游算子发送的数据量和下游算子接收的数据量可能会不相等,这在 Web UI 的显示上可能会困扰用户。细节详见 [FLIP-187](https://cwiki.apache.org/confluence/display/FLINK/FLIP-187%3A+Adaptive+Batch+Job+Scheduler)。 {{< top >}} diff --git a/docs/content.zh/docs/deployment/filesystems/azure.md b/docs/content.zh/docs/deployment/filesystems/azure.md index 73bcf5c6fe5fc..42a0d09ee967f 100644 --- a/docs/content.zh/docs/deployment/filesystems/azure.md +++ b/docs/content.zh/docs/deployment/filesystems/azure.md @@ -61,7 +61,7 @@ abfss://@$.dfs.core.windows.net/@$.blob.core.windows.net/"); // 写入 Azure Blob 存储 -stream.writeAsText("wasb://@$.blob.core.windows.net/") +stream.writeAsText("wasb://@$.blob.core.windows.net/"); // 将 Azure Blob 存储用作 FsStatebackend env.setStateBackend(new FsStateBackend("wasb://@$.blob.core.windows.net/")); diff --git a/docs/content.zh/docs/deployment/filesystems/oss.md b/docs/content.zh/docs/deployment/filesystems/oss.md index 07dfb38396b8a..e1ca862276c58 100644 --- a/docs/content.zh/docs/deployment/filesystems/oss.md +++ b/docs/content.zh/docs/deployment/filesystems/oss.md @@ -46,7 +46,7 @@ oss:/// env.readTextFile("oss:///"); // 写入 OSS bucket -stream.writeAsText("oss:///") +stream.writeAsText("oss:///"); // 将 OSS 用作 FsStatebackend env.setStateBackend(new FsStateBackend("oss:///")); diff --git a/docs/content.zh/docs/deployment/ha/overview.md b/docs/content.zh/docs/deployment/ha/overview.md index c7e516b55b2c0..0cfcd3087f770 100644 --- a/docs/content.zh/docs/deployment/ha/overview.md +++ b/docs/content.zh/docs/deployment/ha/overview.md @@ -76,14 +76,16 @@ Flink 提供了两种高可用服务实现: ## JobResultStore -In order to preserve a job's scheduling status across failover events and prevent erroneous -re-execution of globally terminated (i.e. finished, cancelled or failed) jobs, Flink persists -status of terminated jobs to a filesystem using the JobResultStore. -The JobResultStore allows job results to outlive a finished job, and can be used by -Flink components involved in the recovery of a highly-available cluster in order to -determine whether a job should be subject to recovery. - -The JobResultStore has sensible defaults for its behaviour, such as result storage -location, but these can be [configured]({{< ref "docs/deployment/config#high-availability" >}}). +The JobResultStore is used to archive the final result of a job that reached a globally-terminal +state (i.e. finished, cancelled or failed). The data is stored on a file system (see +[job-result-store.storage-path]({{< ref "docs/deployment/config#job-result-store-storage-path" >}})). +Entries in this store are marked as dirty as long as the corresponding job wasn't cleaned up properly +(artifacts are found in the job's subfolder in [high-availability.storageDir]({{< ref "docs/deployment/config#high-availability-storagedir" >}})). + +Dirty entries are subject to cleanup, i.e. the corresponding job is either cleaned up by Flink at +the moment or will be picked up for cleanup as part of a recovery. The entries will be deleted as +soon as the cleanup succeeds. Check the JobResultStore configuration parameters under +[HA configuration options]({{< ref "docs/deployment/config#high-availability" >}}) for further +details on how to adapt the behavior. {{< top >}} diff --git a/docs/content.zh/docs/deployment/memory/network_mem_tuning.md b/docs/content.zh/docs/deployment/memory/network_mem_tuning.md index 7d80135752296..cccba3bff4be8 100644 --- a/docs/content.zh/docs/deployment/memory/network_mem_tuning.md +++ b/docs/content.zh/docs/deployment/memory/network_mem_tuning.md @@ -126,7 +126,11 @@ Flink 有多个本地缓冲区池 —— 每个输出和输入流对应一个。 在往下游 subtask 发送数据部分时,缓冲区通过汇集 record 来优化网络开销。下游 subtask 应该在接收到完整的 record 后才开始处理它。 -如果缓冲区太小(比如小于一条 record),会因为开销比较大而导致吞吐低。 +If the buffer size is too small, or the buffers are flushed too frequently (`execution.buffer-timeout` configuration parameter), this can lead to decreased throughput +since the per-buffer overhead are significantly higher then per-record overheads in the Flink's runtime. + +As a rule of thumb, we don't recommend thinking about increasing the buffer size, or the buffer timeout unless you can observe a network bottleneck in your real life workload +(downstream operator idling, upstream backpressured, output buffer queue is full, downstream input queue is empty). 如果缓冲区太大,会导致: - 内存使用高 diff --git a/docs/content.zh/docs/deployment/overview.md b/docs/content.zh/docs/deployment/overview.md index df6abeaf97826..f10dab34e10c5 100644 --- a/docs/content.zh/docs/deployment/overview.md +++ b/docs/content.zh/docs/deployment/overview.md @@ -158,6 +158,11 @@ Once a job has reached a globally terminal state of either finished, failed or c external component resources associated with the job are then cleaned up. In the event of a failure when cleaning up a resource, Flink will attempt to retry the cleanup. You can [configure]({{< ref "docs/deployment/config#retryable-cleanup" >}}) the retry strategy used. +Reaching the maximum number of retries without succeeding will leave the job in a dirty state. +Its artifacts would need to be cleaned up manually (see the +[High Availability Services / JobResultStore]({{< ref "docs/deployment/ha/overview#jobresultstore" >}}) +section for further details). Restarting the very same job (i.e. using the same +job ID) will result in the cleanup being restarted without running the job again. There is currently an issue with the cleanup of CompletedCheckpoints that failed to be deleted while subsuming them as part of the usual CompletedCheckpoint management. These artifacts are diff --git a/docs/content.zh/docs/dev/configuration/_index.md b/docs/content.zh/docs/dev/configuration/_index.md new file mode 100644 index 0000000000000..0ad3d6b8a22b5 --- /dev/null +++ b/docs/content.zh/docs/dev/configuration/_index.md @@ -0,0 +1,23 @@ +--- +title: "项目配置" +bookCollapseSection: true +weight: 1 +--- + diff --git a/docs/content.zh/docs/dev/configuration/advanced.md b/docs/content.zh/docs/dev/configuration/advanced.md new file mode 100644 index 0000000000000..b94094732fac6 --- /dev/null +++ b/docs/content.zh/docs/dev/configuration/advanced.md @@ -0,0 +1,106 @@ +--- +title: "高级配置" +weight: 10 +type: docs +--- + + +# 高级配置主题 + +## Flink 依赖剖析 + +Flink 自身由一组类和依赖项组成,这些共同构成了 Flink 运行时的核心,在 Flink 应用程序启动时必须存在,会提供诸如通信协调、网络管理、检查点、容错、API、算子(如窗口)、资源管理等领域的服务。 + +这些核心类和依赖项都打包在 `flink-dist.jar`,可以在下载的发行版 `/lib` 文件夹中找到,也是 Flink 容器镜像的基础部分。您可以将其近似地看作是包含 `String` 和 `List` 等公用类的 Java 核心库。 + +为了保持核心依赖项尽可能小并避免依赖冲突,Flink Core Dependencies 不包含任何连接器或库(如 CEP、SQL、ML),以避免在类路径中有过多的类和依赖项。 + +Flink 发行版的 `/lib` 目录里还有包括常用模块在内的各种 JAR 文件,例如 [执行 Table 作业的必需模块](#Table-依赖剖析) 、一组连接器和 format。默认情况下会自动加载,若要禁止加载只需将它们从 classpath 中的 `/lib` 目录中删除即可。 + +Flink 还在 `/opt` 文件夹下提供了额外的可选依赖项,可以通过移动这些 JAR 文件到 `/lib` 目录来启用这些依赖项。 + +有关类加载的更多细节,请查阅 [Flink 类加载]({{< ref "docs/ops/debugging/debugging_classloading.zh.md" >}})。 + +## Scala 版本 + +不同的 Scala 版本二进制不兼容,所有(传递地)依赖于 Scala 的 Flink 依赖项都以它们构建的 Scala 版本为后缀(如 `flink-streaming-scala_2.12`)。 + +如果您只使用 Flink 的 Java API,您可以使用任何 Scala 版本。如果您使用 Flink 的 Scala API,则需要选择与应用程序的 Scala 匹配的 Scala 版本。 + +有关如何为特定 Scala 版本构建 Flink 的细节,请查阅[构建指南]({{< ref "docs/flinkDev/building" >}}#scala-versions)。 + +2.12.8 之后的 Scala 版本与之前的 2.12.x 版本二进制不兼容,使 Flink 项目无法将其 2.12.x 版本直接升级到 2.12.8 以上。您可以按照[构建指南]({{< ref "docs/flinkDev/building" >}}#scala-versions)在本地为更高版本的 Scala 构建 Flink 。为此,您需要在构建时添加 `-Djapicmp.skip` 以跳过二进制兼容性检查。 + +有关更多细节,请查阅 [Scala 2.12.8 版本说明](https://github.com/scala/scala/releases/tag/v2.12.8)。相关部分指出: + +第二项修改是二进制不兼容的:2.12.8 编译器忽略了由更早版本的 2.12 编译器生成的某些方法。然而我们相信这些方法永远不会被使用,现有的编译代码仍可工作。有关更多详细信息,请查阅[pull request 描述](https://github.com/scala/scala/pull/7469)。 + +## Table 依赖剖析 + +Flink 发行版默认包含执行 Flink SQL 任务的必要 JAR 文件(位于 `/lib` 目录),主要有: + +- `flink-table-api-java-uber-{{< version >}}.jar` → 包含所有的 Java API; +- `flink-table-runtime-{{< version >}}.jar` → 包含 Table 运行时; +- `flink-table-planner-loader-{{< version >}}.jar` → 包含查询计划器。 + +{{< hint warning >}} +以前,这些 JAR 都打包进了 `flink-table.jar`,自从 Flink 1.15 开始,已将其划分成三个 JAR,以允许用户使用 `flink-table-planner-loader-{{< version >}}.jar` 充当 `flink-table-planner{{< scala_version >}}-{{< version >}}.jar`。 +{{< /hint >}} + +虽然 Table Java API 内置于发行版中,但默认情况下不包含 Table Scala API。在 Flink Scala API 中使用格式和连接器时,您需要手动下载这些 JAR 包并将其放到发行版的 `/lib` 文件夹中(推荐),或者将它们打包为 Flink SQL 作业的 uber/fat JAR 包中的依赖项。 + +有关更多细节,请查阅如何[连接外部系统]({{< ref "docs/connectors/table/overview" >}})。 + +### Table Planner 和 Table Planner 加载器 + +从 Flink 1.15 开始,发行版包含两个 planner: + +- `flink-table-planner{{< scala_version >}}-{{< version >}}.jar`, 位于 `/opt` 目录, 包含查询计划器; +- `flink-table-planner-loader-{{< version >}}.jar`, 位于 `/lib` 目录默认被加载, 包含隐藏在单独的 classpath 里的查询计划器 (您无法直接使用 `io.apache.flink.table.planner` 包)。 + +这两个 planner JAR 文件的代码功能相同,但打包方式不同。若使用第一个文件,您必须使用与其相同版本的 Scala;若使用第二个,由于 Scala 已经被打包进该文件里,您不需要考虑 Scala 版本问题。 + +默认情况下,发行版使用 `flink-table-planner-loader`。如果想使用内部查询计划器,您可以换掉 JAR 包(拷贝 `flink-table-planner{{< scala_version >}}.jar` 并复制到发行版的 `/lib` 目录)。请注意,此时会被限制用于 Flink 发行版的 Scala 版本。 + +{{< hint danger >}} +这两个 planner 无法同时存在于 classpath,如果您在 `/lib` 目录同时加载他们,Table 任务将会失败。 +{{< /hint >}} + +{{< hint warning >}} +在即将发布的 Flink 版本中,我们将停止在 Flink 发行版中发布 `flink-table-planner{{< scala_version >}}` 组件。我们强烈建议迁移您的作业/自定义连接器/格式以使用前述 API 模块,而不依赖此内部 planner。如果您需要 planner 中尚未被 API 模块暴露的一些功能,请与社区讨论。 +{{< /hint >}} + +## Hadoop 依赖 + +**一般规则:** 没有必要直接添加 Hadoop 依赖到您的应用程序里,唯一的例外是您通过 [Hadoop 兼容](https://nightlies.apache.org/flink/flink-docs-master/docs/dev/dataset/hadoop_compatibility/) 使用已有的 Hadoop 读写 format。 + +如果您想将 Flink 与 Hadoop 一起使用,您需要有一个包含 Hadoop 依赖项的 Flink 系统,而不是添加 Hadoop 作为应用程序依赖项。换句话说,Hadoop 必须是 Flink 系统本身的依赖,而不是用户代码的依赖。Flink 将使用 `HADOOP_CLASSPATH` 环境变量指定 Hadoop 依赖项,可以这样设置: + +```bash +export HADOOP_CLASSPATH=`hadoop classpath` +``` + +这样设计有两个主要原因: + +- 一些 Hadoop 交互可能在用户应用程序启动之前就发生在 Flink 内核。其中包括为检查点配置 HDFS、通过 Hadoop 的 Kerberos 令牌进行身份验证或在 YARN 上部署; + +- Flink 的反向类加载方式在核心依赖项中隐藏了许多传递依赖项。这不仅适用于 Flink 自己的核心依赖项,也适用于已有的 Hadoop 依赖项。这样,应用程序可以使用相同依赖项的不同版本,而不会遇到依赖项冲突。当依赖树变得非常大时,这非常有用。 + +如果您在 IDE 内开发或测试期间需要 Hadoop 依赖项(比如用于 HDFS 访问),应该限定这些依赖项的使用范围(如 *test* 或 *provided*)。 diff --git a/docs/content.zh/docs/dev/configuration/connector.md b/docs/content.zh/docs/dev/configuration/connector.md new file mode 100644 index 0000000000000..95f131a946609 --- /dev/null +++ b/docs/content.zh/docs/dev/configuration/connector.md @@ -0,0 +1,56 @@ +--- +title: "连接器和格式" +weight: 5 +type: docs +--- + + +# 连接器和格式 + +Flink 应用程序可以通过连接器读取和写入各种外部系统。它支持多种格式,以便对数据进行编码和解码以匹配 Flink 的数据结构。 + +[DataStream]({{< ref "docs/connectors/datastream/overview.zh.md" >}}) 和 [Table API/SQL]({{< ref "docs/connectors/table/overview.zh.md" >}}) 都提供了连接器和格式的概述。 + +## 可用的组件 + +为了使用连接器和格式,您需要确保 Flink 可以访问实现了这些功能的组件。对于 Flink 社区支持的每个连接器,我们在 [Maven Central](https://search.maven.org) 发布了两类组件: + +* `flink-connector-` 这是一个精简 JAR,仅包括连接器代码,但不包括最终的第三方依赖项; +* `flink-sql-connector-` 这是一个包含连接器第三方依赖项的 uber JAR; + +这同样适用于格式。请注意,某些连接器可能没有相应的 `flink-sql-connector-` 组件,因为它们不需要第三方依赖项。 + +{{< hint info >}} +uber/fat JAR 主要与[SQL 客户端]({{< ref "docs/dev/table/sqlClient" >}})一起使用,但您也可以在任何 DataStream/Table 应用程序中使用它们。 +{{< /hint >}} + +## 使用组件 + +为了使用连接器/格式模块,您可以: + +* 把精简 JAR 及其传递依赖项打包进您的作业 JAR; +* 把 uber JAR 打包进您的作业 JAR; +* 把 uber JAR 直接复制到 Flink 发行版的 `/lib` 文件夹内; + +关于打包依赖项,请查看 [Maven]({{< ref "docs/dev/configuration/maven" >}}) 和 [Gradle]({{< ref "docs/dev/configuration/gradle" >}}) 指南。有关 Flink 发行版的参考,请查看[Flink 依赖剖析]({{< ref "docs/dev/configuration/overview" >}}#Flink-依赖剖析)。 + +{{< hint info >}} +决定是打成 uber JAR、精简 JAR 还是仅在发行版包含依赖项取决于您和您的使用场景。如果您使用 uber JAR,您将对作业里的依赖项版本有更多的控制权;如果您使用精简 JAR,由于您可以在不更改连接器版本的情况下更改版本(允许二进制兼容),您将对传递依赖项有更多的控制权;如果您直接在 Flink 发行版的 `/lib` 目录里内嵌连接器 uber JAR,您将能够在一处控制所有作业的连接器版本。 +{{< /hint >}} diff --git a/docs/content.zh/docs/dev/configuration/gradle.md b/docs/content.zh/docs/dev/configuration/gradle.md new file mode 100644 index 0000000000000..a7455060b51ef --- /dev/null +++ b/docs/content.zh/docs/dev/configuration/gradle.md @@ -0,0 +1,92 @@ +--- +title: "使用 Gradle" +weight: 3 +type: docs +--- + + +# 如何使用 Gradle 配置您的项目 + +您可能需要一个构建工具来配置您的 Flink 项目,本指南将向您展示如何使用 [Gradle](https://gradle.org) 执行此操作。Gradle 是一个开源的通用构建工具,可用于在开发过程中自动化执行任务。 + +## 要求 + +- Gradle 7.x +- Java 11 + +## 将项目导入 IDE + +创建[项目目录和文件]({{< ref "docs/dev/configuration/overview#getting-started" >}})后,我们建议您将此项目导入到 IDE 进行开发和测试。 + +IntelliJ IDEA 通过 `Gradle` 插件支持 Gradle 项目。 + +Eclipse 通过 [Eclipse Buildship](https://projects.eclipse.org/projects/tools.buildship) 插件执行此操作(确保在导入向导的最后一步中指定 Gradle 版本 >= 3.0,`shadow` 插件会用到它)。您还可以使用 [Gradle 的 IDE 集成](https://docs.gradle.org/current/userguide/userguide.html#ide-integration) 来使用 Gradle 创建项目文件。 + +**注意:** Java 的默认 JVM 堆大小对于 Flink 来说可能太小,您应该手动增加它。在 Eclipse 中,选中 `Run Configurations -> Arguments` 并在 `VM Arguments` 框里填上:`-Xmx800m`。在 IntelliJ IDEA 中,推荐选中 `Help | Edit Custom VM Options` 菜单修改 JVM 属性。详情请查阅[本文](https://intellij-support.jetbrains.com/hc/en-us/articles/206544869-Configuring-JVM-options-and-platform-properties)。 + +**关于 IntelliJ 的注意事项:** 要使应用程序在 IntelliJ IDEA 中运行,需要在运行配置中的 `Include dependencies with "Provided" scope` 打勾。如果此选项不可用(可能是由于使用了较旧的 IntelliJ IDEA 版本),可创建一个调用应用程序 `main()` 方法的测试用例。 + +## 构建项目 + +如果您想 __构建/打包__ 您的项目,请转到您的项目目录并运行 '`gradle clean shadowJar`' 命令。您将 __找到一个 JAR 文件__,其中包含您的应用程序,还有已作为依赖项添加到应用程序的连接器和库:`build/libs/--all.jar`。 + +__注意:__ 如果您使用不同于 *StreamingJob* 的类作为应用程序的主类/入口点,我们建议您对 `build.gradle` 文件里的 `mainClassName` 配置进行相应的修改。这样,Flink 可以通过 JAR 文件运行应用程序,而无需额外指定主类。 + +## 向项目添加依赖项 + +在 `build.gradle` 文件的 dependencies 块中配置依赖项 + +例如,如果您使用我们的 Gradle 构建脚本或快速启动脚本创建了项目,如下所示,可以将 Kafka 连接器添加为依赖项: + +**build.gradle** + +```gradle +... +dependencies { + ... + flinkShadowJar "org.apache.flink:flink-connector-kafka:${flinkVersion}" + ... +} +... +``` + +**重要提示:** 请注意,应将所有这些(核心)依赖项的生效范围置为 [*provided*](https://maven.apache.org/guides/introduction/introduction-to-dependency-mechanism.html#dependency-scope)。这意味着需要对它们进行编译,但不应将它们打包进项目生成的应用程序 JAR 文件中。如果不设置为 *provided*,最好的情况是生成的 JAR 变得过大,因为它还包含所有 Flink 核心依赖项。最坏的情况是添加到应用程序 JAR 文件中的 Flink 核心依赖项与您自己的一些依赖项的版本冲突(通常通过反向类加载来避免)。 + +要将依赖项正确地打包进应用程序 JAR 中,必须把应用程序依赖项的生效范围设置为 *compile* 。 + +## 打包应用程序 + +在部署应用到 Flink 环境之前,您需要根据使用场景用不同的方式打包 Flink 应用程序。 + +如果您想为 Flink 作业创建 JAR 并且只使用 Flink 依赖而不使用任何第三方依赖(比如使用 JSON 格式的文件系统连接器),您不需要创建一个 uber/fat JAR 或将任何依赖打进包。 + +您可以使用 `gradle clean installDist` 命令,如果您使用的是 [Gradle Wrapper](https://docs.gradle.org/current/userguide/gradle_wrapper.html) ,则用 `./gradlew clean installDist`。 + +如果您想为 Flink 作业创建 JAR 并使用未内置在 Flink 发行版中的外部依赖项,您可以将它们添加到发行版的类路径中,或者将它们打包进您的 uber/fat 应用程序 JAR 中。 + +您可以使用该命令 `gradle clean installShadowDist`,该命令将在 `/build/install/yourProject/lib` 目录生成一个 fat JAR。如果您使用的是 [Gradle Wrapper](https://docs.gradle.org/current/userguide/gradle_wrapper.html) ,则用 `./gradlew clean installShadowDist`。 + +您可以将生成的 uber/fat JAR 提交到本地或远程集群: + +```sh +bin/flink run -c org.example.MyJob myFatJar.jar +``` + +要了解有关如何部署 Flink 作业的更多信息,请查看[部署指南]({{< ref "docs/deployment/cli" >}})。 diff --git a/docs/content.zh/docs/dev/configuration/maven.md b/docs/content.zh/docs/dev/configuration/maven.md new file mode 100644 index 0000000000000..bc1a71578257b --- /dev/null +++ b/docs/content.zh/docs/dev/configuration/maven.md @@ -0,0 +1,142 @@ +--- +title: "使用 Maven" +weight: 2 +type: docs +--- + + +# 如何使用 Maven 配置您的项目 + +本指南将向您展示如何使用 [Maven](https://maven.apache.org) 配置 Flink 作业项目,Maven是 由 Apache Software Foundation 开源的自动化构建工具,使您能够构建、发布和部署项目。您可以使用它来管理软件项目的整个生命周期。 + +## 要求 + +- Maven 3.0.4 (or higher) +- Java 11 + +## 将项目导入 IDE + +创建[项目目录和文件]({{< ref "docs/dev/configuration/overview#getting-started" >}})后,我们建议您将此项目导入到 IDE 进行开发和测试。 + +IntelliJ IDEA 支持开箱即用的 Maven 项目。Eclipse 提供了 [m2e 插件](http://www.eclipse.org/m2e/) 来[导入 Maven 项目](http://books.sonatype.com/m2eclipse-book/reference/creating-sect-importing-projects.html#fig-creating-import)。 + +**注意:** Java 的默认 JVM 堆大小对于 Flink 来说可能太小,您应该手动增加它。在 Eclipse 中,选中 `Run Configurations -> Arguments` 并在 `VM Arguments` 框里填上:`-Xmx800m`。在 IntelliJ IDEA 中,推荐选中 `Help | Edit Custom VM Options` 菜单修改 JVM 属性。详情请查阅[本文](https://intellij-support.jetbrains.com/hc/en-us/articles/206544869-Configuring-JVM-options-and-platform-properties)。 + +**关于 IntelliJ 的注意事项:** 要使应用程序在 IntelliJ IDEA 中运行,需要在运行配置中的 Include dependencies with "Provided" scope` 打勾。如果此选项不可用(可能是由于使用了较旧的 IntelliJ IDEA 版本),可创建一个调用应用程序 `main()` 方法的测试用例。 + +## 构建项目 + +如果您想 __构建/打包__ 您的项目,请转到您的项目目录并运行 '`mvn clean package`' 命令。您将 __找到一个 JAR 文件__,其中包含您的应用程序(还有已作为依赖项添加到应用程序的连接器和库):`target/-.jar`。 + +__注意:__ 如果您使用不同于 `DataStreamJob` 的类作为应用程序的主类/入口点,我们建议您对 `pom.xml` 文件里的 `mainClassName` 配置进行相应的修改。这样,Flink 可以通过 JAR 文件运行应用程序,而无需额外指定主类。 + +## 向项目添加依赖项 + +打开您项目目录的 `pom.xml`,在 `dependencies` 标签内添加依赖项。 + +例如,您可以用如下方式添加 Kafka 连接器依赖: + +```xml + + + + org.apache.flink + flink-connector-kafka + {{< version >}} + + + +``` + +然后在命令行执行 `mvn install`。 + +当您在由 `Java Project Template`、`Scala Project Template` 或 Gradle 创建出来的项目里,运行 `mvn clean package` 会自动将应用程序依赖打包进应用程序 JAR。对于不是通过这些模板创建的项目,我们建议使用 Maven Shade 插件以将所有必需的依赖项打包进应用程序 jar。 + +**重要提示:** 请注意,应将所有这些(核心)依赖项的生效范围置为 [*provided*](https://maven.apache.org/guides/introduction/introduction-to-dependency-mechanism.html#dependency-scope)。这意味着需要对它们进行编译,但不应将它们打包进项目生成的应用程序 JAR 文件中。如果不设置为 *provided*,最好的情况是生成的 JAR 变得过大,因为它还包含所有 Flink 核心依赖项。最坏的情况是添加到应用程序 JAR 文件中的 Flink 核心依赖项与您自己的一些依赖项的版本冲突(通常通过反向类加载来避免)。 + +要将依赖项正确地打包进应用程序 JAR 中,必须把应用程序依赖项的生效范围设置为 *compile* 。 + +## 打包应用程序 + +在部署应用到 Flink 环境之前,您需要根据使用场景用不同的方式打包 Flink 应用程序。 + +如果您想为 Flink 作业创建 JAR 并且只使用 Flink 依赖而不使用任何第三方依赖(比如使用 JSON 格式的文件系统连接器),您不需要创建一个 uber/fat JAR 或将任何依赖打进包。 + +如果您想为 Flink 作业创建 JAR 并使用未内置在 Flink 发行版中的外部依赖项,您可以将它们添加到发行版的类路径中,或者将它们打包进您的 uber/fat 应用程序 JAR 中。 + +您可以将生成的 uber/fat JAR 提交到本地或远程集群: + +```sh +bin/flink run -c org.example.MyJob myFatJar.jar +``` + +要了解有关如何部署 Flink 作业的更多信息,请查看[部署指南]({{< ref "docs/deployment/cli" >}})。 + +## 创建包含依赖项的 uber/fat JAR 的模板 + +为构建一个包含所有必需的连接器、 类库依赖项的应用程序 JAR,您可以使用如下 shade 插件定义: + +```xml + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.1 + + + package + + shade + + + + + com.google.code.findbugs:jsr305 + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + my.programs.main.clazz + + + + + + + + + +``` + +[Maven shade 插件](https://maven.apache.org/plugins/maven-shade-plugin/index.html) 默认会包含所有的生效范围是 "runtime" 或 "compile" 的依赖项。 diff --git a/docs/content.zh/docs/dev/configuration/overview.md b/docs/content.zh/docs/dev/configuration/overview.md new file mode 100644 index 0000000000000..84f20abd67858 --- /dev/null +++ b/docs/content.zh/docs/dev/configuration/overview.md @@ -0,0 +1,206 @@ +--- +title: "概览" +weight: 1 +type: docs +aliases: +- /dev/project-configuration.html +- /start/dependencies.html +- /getting-started/project-setup/dependencies.html +- /quickstart/java_api_quickstart.html +- /dev/projectsetup/java_api_quickstart.html +- /dev/linking_with_flink.html +- /dev/linking.html +- /dev/projectsetup/dependencies.html +- /dev/projectsetup/java_api_quickstart.html +- /getting-started/project-setup/java_api_quickstart.html +- /dev/getting-started/project-setup/scala_api_quickstart.html +- /getting-started/project-setup/scala_api_quickstart.html +- /quickstart/scala_api_quickstart.html +--- + + +# 项目配置 + +本节将向您展示如何通过流行的构建工具 ([Maven]({{< ref "docs/dev/configuration/maven" >}})、[Gradle]({{< ref "docs/dev/configuration/gradle" >}})) 配置您的项目,必要的依赖项(比如[连接器和格式]({{< ref "docs/dev/configuration/connector" >}})),以及覆盖一些[高级]({{< ref "docs/dev/configuration/advanced" >}})配置主题。 + +每个 Flink 应用程序都依赖于一组 Flink 库。应用程序至少依赖于 Flink API,此外还依赖于某些连接器库(比如 Kafka、Cassandra),以及用户开发的自定义的数据处理逻辑所需要的第三方依赖项。 + +## 开始 + +要开始使用 Flink 应用程序,请使用以下命令、脚本和模板来创建 Flink 项目。 + +{{< tabs "creating project" >}} +{{< tab "Maven" >}} + +您可以使用如下的 Maven 命令或快速启动脚本,基于[原型](https://maven.apache.org/guides/introduction/introduction-to-archetypes.html)创建一个项目。 + +### Maven 命令 +```bash +$ mvn archetype:generate \ + -DarchetypeGroupId=org.apache.flink \ + -DarchetypeArtifactId=flink-quickstart-java \ + -DarchetypeVersion={{< version >}} +``` +这允许您命名新建的项目,而且会交互式地询问 groupId、artifactId、package 的名字。 + +### 快速启动脚本 +```bash +$ curl https://flink.apache.org/q/quickstart.sh | bash -s {{< version >}} +``` + +{{< /tab >}} +{{< tab "Gradle" >}} +您可以使用如下的 Gradle 构建脚本或快速启动脚本创建一个项目。 + +### Gradle 构建脚本 + +请在脚本的所在目录执行 `gradle` 命令来执行这些构建配置脚本。 + +**build.gradle** + +```gradle +plugins { + id 'java' + id 'application' + // shadow plugin to produce fat JARs + id 'com.github.johnrengelman.shadow' version '7.1.2' +} +// artifact properties +group = 'org.quickstart' +version = '0.1-SNAPSHOT' +mainClassName = 'org.quickstart.StreamingJob' +mainClassName = 'org.quickstart.StreamingJob' +description = """Flink Quickstart Job""" +ext { + javaVersion = '1.8' + flinkVersion = '{{< version >}}' + slf4jVersion = '1.7.32' + log4jVersion = '2.17.1' +} +sourceCompatibility = javaVersion +targetCompatibility = javaVersion +tasks.withType(JavaCompile) { + options.encoding = 'UTF-8' +} +applicationDefaultJvmArgs = ["-Dlog4j.configurationFile=log4j2.properties"] + +// declare where to find the dependencies of your project +repositories { + mavenCentral() +} +// NOTE: We cannot use "compileOnly" or "shadow" configurations since then we could not run code +// in the IDE or with "gradle run". We also cannot exclude transitive dependencies from the +// shadowJar yet (see https://github.com/johnrengelman/shadow/issues/159). +// -> Explicitly define the // libraries we want to be included in the "flinkShadowJar" configuration! +configurations { + flinkShadowJar // dependencies which go into the shadowJar + // always exclude these (also from transitive dependencies) since they are provided by Flink + flinkShadowJar.exclude group: 'org.apache.flink', module: 'force-shading' + flinkShadowJar.exclude group: 'com.google.code.findbugs', module: 'jsr305' + flinkShadowJar.exclude group: 'org.slf4j' + flinkShadowJar.exclude group: 'org.apache.logging.log4j' +} +// declare the dependencies for your production and test code +dependencies { + // -------------------------------------------------------------- + // Compile-time dependencies that should NOT be part of the + // shadow (uber) jar and are provided in the lib folder of Flink + // -------------------------------------------------------------- + implementation "org.apache.flink:flink-streaming-java:${flinkVersion}" + implementation "org.apache.flink:flink-clients:${flinkVersion}" + // -------------------------------------------------------------- + // Dependencies that should be part of the shadow jar, e.g. + // connectors. These must be in the flinkShadowJar configuration! + // -------------------------------------------------------------- + //flinkShadowJar "org.apache.flink:flink-connector-kafka:${flinkVersion}" + runtimeOnly "org.apache.logging.log4j:log4j-api:${log4jVersion}" + runtimeOnly "org.apache.logging.log4j:log4j-core:${log4jVersion}" + runtimeOnly "org.apache.logging.log4j:log4j-slf4j-impl:${log4jVersion}" + runtimeOnly "org.slf4j:slf4j-log4j12:${slf4jVersion}" + // Add test dependencies here. + // testCompile "junit:junit:4.12" +} +// make compileOnly dependencies available for tests: +sourceSets { + main.compileClasspath += configurations.flinkShadowJar + main.runtimeClasspath += configurations.flinkShadowJar + test.compileClasspath += configurations.flinkShadowJar + test.runtimeClasspath += configurations.flinkShadowJar + javadoc.classpath += configurations.flinkShadowJar +} +run.classpath = sourceSets.main.runtimeClasspath + +shadowJar { + configurations = [project.configurations.flinkShadowJar] +} +``` + +**settings.gradle** + +```gradle +rootProject.name = 'quickstart' +``` + +### 快速启动脚本 + +```bash +bash -c "$(curl https://flink.apache.org/q/gradle-quickstart.sh)" -- {{< version >}} {{< scala_version >}} +``` +{{< /tab >}} +{{< /tabs >}} + +## 需要哪些依赖项? + +要开始一个 Flink 作业,您通常需要如下依赖项: + +* Flink API, 用来开发您的作业 +* [连接器和格式]({{< ref "docs/dev/configuration/connector" >}}), 以将您的作业与外部系统集成 +* [测试实用程序]({{< ref "docs/dev/configuration/testing" >}}), 以测试您的作业 + +除此之外,若要开发自定义功能,您还要添加必要的第三方依赖项。 + +### Flink API + +Flink提供了两大 API:[Datastream API]({{< ref "docs/dev/datastream/overview" >}}) 和 [Table API & SQL]({{< ref "docs/dev/table/overview" >}}),它们可以单独使用,也可以混合使用,具体取决于您的使用场景: + +| 您要使用的 API | 您需要添加的依赖项 | +|-----------------------------------------------------------------------------------|-----------------------------------------------------| +| [DataStream]({{< ref "docs/dev/datastream/overview" >}}) | `flink-streaming-java` | +| [DataStream Scala 版]({{< ref "docs/dev/datastream/scala_api_extensions" >}}) | `flink-streaming-scala{{< scala_version >}}` | +| [Table API]({{< ref "docs/dev/table/common" >}}) | `flink-table-api-java` | +| [Table API Scala 版]({{< ref "docs/dev/table/common" >}}) | `flink-table-api-scala{{< scala_version >}}` | +| [Table API + DataStream]({{< ref "docs/dev/table/data_stream_api" >}}) | `flink-table-api-java-bridge` | +| [Table API + DataStream Scala 版]({{< ref "docs/dev/table/data_stream_api" >}}) | `flink-table-api-scala-bridge{{< scala_version >}}` | + +您只需将它们包含在您的构建工具脚本/描述符中,就可以开发您的作业了! + +## 运行和打包 + +如果您想通过简单地执行主类来运行你的作业,您需要 classpath 里有 `flink-runtime`。对于 Table API 程序,您还需要 `flink-table-runtime` 和 `flink-table-planner-loader`。 + +根据经验,我们**建议**将应用程序代码及其所有必需的依赖项打包进一个 fat/uber JAR 中。这包括打包您作业用到的连接器、格式和第三方依赖项。此规则**不适用于** Java API、DataStream Scala API 以及前面提到的运行时模块,它们已经由 Flink 本身提供,**不应**包含在作业的 uber JAR 中。您可以把该作业 JAR 提交到已经运行的 Flink 集群,也可以轻松将其添加到 Flink 应用程序容器镜像中,而无需修改发行版。 + +## 下一步是什么? + +* 要开发您的作业,请查阅 [DataStream API]({{< ref "docs/dev/datastream/overview" >}}) 和 [Table API & SQL]({{< ref "docs/dev/table/overview" >}}); +* 关于如何使用特定的构建工具打包您的作业的更多细节,请查阅如下指南: + * [Maven]({{< ref "docs/dev/configuration/maven" >}}) + * [Gradle]({{< ref "docs/dev/configuration/gradle" >}}) +* 关于项目配置的高级内容,请查阅[高级主题]({{< ref "docs/dev/configuration/advanced" >}})部分。 diff --git a/docs/content.zh/docs/dev/configuration/testing.md b/docs/content.zh/docs/dev/configuration/testing.md new file mode 100644 index 0000000000000..33e0e9e8f5a8e --- /dev/null +++ b/docs/content.zh/docs/dev/configuration/testing.md @@ -0,0 +1,49 @@ +--- +title: "测试的依赖项" +weight: 6 +type: docs +--- + + +# 用于测试的依赖项 + +Flink 提供了用于测试作业的实用程序,您可以将其添加为依赖项。 + +## DataStream API 测试 + +如果要为使用 DataStream API 构建的作业开发测试用例,则需要添加以下依赖项: + +{{< artifact_tabs flink-test-utils withTestScope >}} + +在各种测试实用程序中,该模块提供了 `MiniCluster` (一个可配置的轻量级 Flink 集群,能在 JUnit 测试中运行),可以直接执行作业。 + +有关如何使用这些实用程序的更多细节,请查看 [DataStream API 测试]({{< ref "docs/dev/datastream/testing" >}})。 + +## Table API 测试 + +如果您想在您的 IDE 中本地测试 Table API 和 SQL 程序,除了前述提到的 `flink-test-utils` 之外,您还要添加以下依赖项: + +{{< artifact_tabs flink-table-test-utils withTestScope >}} + +这将自动引入查询计划器和运行时,分别用于计划和执行查询。 + +{{< hint info >}} +`flink-table-test-utils` 模块已在 Flink 1.15 中引入,目前被认为是实验性的。 +{{< /hint >}} diff --git a/docs/content.zh/docs/dev/dataset/examples.md b/docs/content.zh/docs/dev/dataset/examples.md index e543ca1ca00d3..08b23b1635f09 100644 --- a/docs/content.zh/docs/dev/dataset/examples.md +++ b/docs/content.zh/docs/dev/dataset/examples.md @@ -135,7 +135,7 @@ ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); // 通过解析一个CSV文件来获取每个页面原始的rank值 DataSet> pagesWithRanks = env.readCsvFile(pagesInputPath) - .types(Long.class, Double.class) + .types(Long.class, Double.class); // 链接被编码为邻接表: (page-id, Array(neighbor-ids)) DataSet> pageLinkLists = getLinksDataSet(env); diff --git a/docs/content.zh/docs/dev/dataset/hadoop_compatibility.md b/docs/content.zh/docs/dev/dataset/hadoop_compatibility.md index 18851fab010a7..737a78a6964b6 100644 --- a/docs/content.zh/docs/dev/dataset/hadoop_compatibility.md +++ b/docs/content.zh/docs/dev/dataset/hadoop_compatibility.md @@ -142,7 +142,7 @@ The following example shows how to use Hadoop's `TextOutputFormat`. ```java // Obtain the result we want to emit -DataSet> hadoopResult = [...] +DataSet> hadoopResult = [...]; // Set up the Hadoop TextOutputFormat. HadoopOutputFormat hadoopOF = @@ -198,7 +198,7 @@ The following example shows how to use Hadoop `Mapper` and `Reducer` functions. ```java // Obtain data to process somehow. -DataSet> text = [...] +DataSet> text = [...]; DataSet> result = text // use Hadoop Mapper (Tokenizer) as MapFunction diff --git a/docs/content.zh/docs/dev/dataset/iterations.md b/docs/content.zh/docs/dev/dataset/iterations.md index c563e1d373d46..194d341b77de3 100644 --- a/docs/content.zh/docs/dev/dataset/iterations.md +++ b/docs/content.zh/docs/dev/dataset/iterations.md @@ -171,7 +171,7 @@ IterationState solution = getInitialSolution(); while (!terminationCriterion()) { (delta, workset) = step(workset, solution); - solution.update(delta) + solution.update(delta); } setFinalState(solution); diff --git a/docs/content.zh/docs/dev/dataset/overview.md b/docs/content.zh/docs/dev/dataset/overview.md index 6b51e8e754fce..b23771eb50794 100644 --- a/docs/content.zh/docs/dev/dataset/overview.md +++ b/docs/content.zh/docs/dev/dataset/overview.md @@ -281,7 +281,7 @@ It removes the duplicate entries from the input DataSet, with respect to all fie {{< tabs "distinct" >}} {{< tab "Java" >}} ```java -data.distinct() +data.distinct(); ``` {{< /tab >}} {{< tab "Scala" >}} @@ -431,7 +431,7 @@ Produces the union of two data sets. {{< tabs "union" >}} {{< tab "Java" >}} ```java -data.union(data2) +data.union(data2); ``` {{< /tab >}} {{< tab "Scala" >}} @@ -450,7 +450,7 @@ Only Map-like transformations may follow a rebalance transformation. {{< tab "Java" >}} ```java DataSet data1 = // [...] -DataSet> result = data1.rebalance().map(...) +DataSet> result = data1.rebalance().map(...); ``` {{< /tab >}} {{< tab "Scala" >}} @@ -651,7 +651,7 @@ The simplest case is grouping Tuples on one or more fields of the Tuple: {{< tab "Java" >}} ```java DataSet> input = // [...] -UnsortedGrouping,Tuple> keyed = input.groupBy(0) +UnsortedGrouping,Tuple> keyed = input.groupBy(0); ``` {{< /tab >}} {{< tab "Scala" >}} @@ -668,7 +668,7 @@ Tuples are grouped on the first field (the one of Integer type). {{< tab "Java" >}} ```java DataSet> input = // [...] -UnsortedGrouping,Tuple> keyed = input.groupBy(0,1) +UnsortedGrouping,Tuple> keyed = input.groupBy(0,1); ``` {{< /tab >}} {{< tab "Scala" >}} @@ -707,7 +707,7 @@ public class WC { public int count; } DataSet words = // [...] -DataSet wordCounts = words.groupBy("word") +DataSet wordCounts = words.groupBy("word"); ``` {{< /tab >}} {{< tab "Scala" >}} @@ -1394,11 +1394,11 @@ final ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); DataSet myInts = env.fromElements(1, 2, 3, 4, 5); // Create a DataSet from any Java collection -List> data = ... +List> data = ...; DataSet> myTuples = env.fromCollection(data); // Create a DataSet from an Iterator -Iterator longIt = ... +Iterator longIt = ...; DataSet myLongs = env.fromCollection(longIt, Long.class); ``` {{< /tab >}} @@ -1496,14 +1496,14 @@ The distributed cache is used as follows: ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); // register a file from HDFS -env.registerCachedFile("hdfs:///path/to/your/file", "hdfsFile") +env.registerCachedFile("hdfs:///path/to/your/file", "hdfsFile"); // register a local executable file (script, executable, ...) -env.registerCachedFile("file:///path/to/exec/file", "localExecFile", true) +env.registerCachedFile("file:///path/to/exec/file", "localExecFile", true); // define your program and execute ... -DataSet input = ... +DataSet input = ...; DataSet result = input.map(new MyMapper()); ... env.execute(); diff --git a/docs/content.zh/docs/dev/dataset/transformations.md b/docs/content.zh/docs/dev/dataset/transformations.md index 0edbb8fbda884..d58a38226ad89 100644 --- a/docs/content.zh/docs/dev/dataset/transformations.md +++ b/docs/content.zh/docs/dev/dataset/transformations.md @@ -204,7 +204,7 @@ DataSet> out = in.project(2,0); Note that the Java compiler cannot infer the return type of `project` operator. This can cause a problem if you call another operator on a result of `project` operator such as: ```java -DataSet> ds = .... +DataSet> ds = ....; DataSet> ds2 = ds.project(0).distinct(0); ``` diff --git a/docs/content.zh/docs/dev/datastream/_index.md b/docs/content.zh/docs/dev/datastream/_index.md index 1a3281350eb61..5e035cc14d0e3 100644 --- a/docs/content.zh/docs/dev/datastream/_index.md +++ b/docs/content.zh/docs/dev/datastream/_index.md @@ -1,7 +1,7 @@ --- title: DataStream API bookCollapseSection: true -weight: 1 +weight: 2 --- \ No newline at end of file +--> diff --git a/docs/content.zh/docs/dev/datastream/application_parameters.md b/docs/content.zh/docs/dev/datastream/application_parameters.md index 8d00b6ef39d63..a9ad2c17739da 100644 --- a/docs/content.zh/docs/dev/datastream/application_parameters.md +++ b/docs/content.zh/docs/dev/datastream/application_parameters.md @@ -84,7 +84,7 @@ ParameterTool parameters = // ... parameter.getRequired("input"); parameter.get("output", "myDefaultValue"); parameter.getLong("expectedCount", -1L); -parameter.getNumberOfParameters() +parameter.getNumberOfParameters(); // .. there are more methods available. ``` diff --git a/docs/content.zh/docs/dev/datastream/execution/parallel.md b/docs/content.zh/docs/dev/datastream/execution/parallel.md index bd5f336be357f..d30495c0eb802 100644 --- a/docs/content.zh/docs/dev/datastream/execution/parallel.md +++ b/docs/content.zh/docs/dev/datastream/execution/parallel.md @@ -46,7 +46,7 @@ under the License. ```java final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); -DataStream text = [...] +DataStream text = [...]; DataStream> wordCounts = text .flatMap(new LineSplitter()) .keyBy(value -> value.f0) @@ -87,8 +87,8 @@ env.execute("Word Count Example") final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(3); -DataStream text = [...] -DataStream> wordCounts = [...] +DataStream text = [...]; +DataStream> wordCounts = [...]; wordCounts.print(); env.execute("Word Count Example"); diff --git a/docs/content.zh/docs/dev/datastream/execution_mode.md b/docs/content.zh/docs/dev/datastream/execution_mode.md index e0655b707f9c4..2cf146a0c537c 100644 --- a/docs/content.zh/docs/dev/datastream/execution_mode.md +++ b/docs/content.zh/docs/dev/datastream/execution_mode.md @@ -201,11 +201,11 @@ Checkpointing 用于故障恢复的特点之一是,在发生故障时,Flink `批`模式下的行为变化: -* “滚动"操作,如 [reduce()]({{< ref "docs/dev/datastream/operators/overview" >}}#reduce) 或 [sum()]({{< ref "docs/dev/datastream/operators/overview" >}}#aggregations),会对`流`模式下每一条新记录发出增量更新。在`批`模式下,这些操作不是"滚动”。它们只发出最终结果。 +* “滚动"操作,如 [reduce()]({{< ref "docs/dev/datastream/operators/overview" >}}#reduce) 或 sum(),会对`流`模式下每一条新记录发出增量更新。在`批`模式下,这些操作不是"滚动”。它们只发出最终结果。 `批`模式下不支持的: -* [Checkpointing]({{< ref "docs/concepts/stateful-stream-processing" >}}#stateful-stream-processing) 和任何依赖于 checkpointing 的操作都不支持。 +* [Checkpointing]({{< ref "docs/concepts/stateful-stream-processing" >}}#checkpointing) 和任何依赖于 checkpointing 的操作都不支持。 * [迭代(Iterations)]({{< ref "docs/dev/datastream/operators/overview" >}}#iterate) 自定义算子应谨慎执行,否则可能会有不恰当的行为。更多细节请参见下面的补充说明。 diff --git a/docs/content.zh/docs/dev/datastream/experimental.md b/docs/content.zh/docs/dev/datastream/experimental.md index 90212a5364c99..aaeae8b679720 100644 --- a/docs/content.zh/docs/dev/datastream/experimental.md +++ b/docs/content.zh/docs/dev/datastream/experimental.md @@ -62,7 +62,7 @@ Code example: {{< tab "Java" >}} ```java StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); -DataStreamSource source = ... +DataStreamSource source = ...; DataStreamUtils.reinterpretAsKeyedStream(source, (in) -> in, TypeInformation.of(Integer.class)) .window(TumblingEventTimeWindows.of(Time.seconds(1))) .reduce((a, b) -> a + b) diff --git a/docs/content.zh/docs/dev/datastream/fault-tolerance/checkpointing.md b/docs/content.zh/docs/dev/datastream/fault-tolerance/checkpointing.md index faecd2c417543..650d735e76860 100644 --- a/docs/content.zh/docs/dev/datastream/fault-tolerance/checkpointing.md +++ b/docs/content.zh/docs/dev/datastream/fault-tolerance/checkpointing.md @@ -90,7 +90,7 @@ env.getCheckpointConfig().setMinPauseBetweenCheckpoints(500); env.getCheckpointConfig().setCheckpointTimeout(60000); // 允许两个连续的 checkpoint 错误 -env.getCheckpointConfig().setTolerableCheckpointFailureNumber(2) +env.getCheckpointConfig().setTolerableCheckpointFailureNumber(2); // 同一时间只允许一个 checkpoint 进行 env.getCheckpointConfig().setMaxConcurrentCheckpoints(1); diff --git a/docs/content.zh/docs/dev/datastream/fault-tolerance/queryable_state.md b/docs/content.zh/docs/dev/datastream/fault-tolerance/queryable_state.md index 982854e3f8757..4a9a90845a69d 100644 --- a/docs/content.zh/docs/dev/datastream/fault-tolerance/queryable_state.md +++ b/docs/content.zh/docs/dev/datastream/fault-tolerance/queryable_state.md @@ -98,7 +98,7 @@ QueryableStateStream asQueryableState( 返回的 `QueryableStateStream` 可以被视作一个sink,而且**不能再**被进一步转换。在内部实现上,一个 `QueryableStateStream` 被转换成一个 operator,使用输入的数据来更新 queryable state。state 如何更新是由 `asQueryableState` 提供的 `StateDescriptor` 来决定的。在下面的代码中, keyed stream 的所有数据将会通过 `ValueState.update(value)` 来更新状态: ```java -stream.keyBy(value -> value.f0).asQueryableState("query-name") +stream.keyBy(value -> value.f0).asQueryableState("query-name"); ``` 这个行为类似于 Scala API 中的 `flatMapWithState`。 @@ -143,7 +143,7 @@ descriptor.setQueryable("query-name"); // queryable state name ``` -关于依赖的更多信息, 可以参考如何 [配置 Flink 项目]({{< ref "docs/dev/datastream/project-configuration" >}}). +关于依赖的更多信息, 可以参考如何[配置 Flink 项目]({{< ref "docs/dev/configuration/overview" >}})。 `QueryableStateClient` 将提交你的请求到内部代理,代理会处理请求并返回结果。客户端的初始化只需要提供一个有效的 `TaskManager` 主机名 (每个 task manager 上都运行着一个 queryable state 代理),以及代理监听的端口号。关于如何配置代理以及端口号可以参考 [Configuration Section](#configuration). diff --git a/docs/content.zh/docs/dev/datastream/fault-tolerance/serialization/types_serialization.md b/docs/content.zh/docs/dev/datastream/fault-tolerance/serialization/types_serialization.md index ee1055ecf1479..c80f7b4619594 100644 --- a/docs/content.zh/docs/dev/datastream/fault-tolerance/serialization/types_serialization.md +++ b/docs/content.zh/docs/dev/datastream/fault-tolerance/serialization/types_serialization.md @@ -499,7 +499,7 @@ env.getConfig().enableForceKryo(); If Kryo is not able to serialize your POJO, you can add a custom serializer to Kryo, using ```java -env.getConfig().addDefaultKryoSerializer(Class type, Class> serializerClass) +env.getConfig().addDefaultKryoSerializer(Class type, Class> serializerClass); ``` There are different variants of these methods available. diff --git a/docs/content.zh/docs/dev/datastream/fault-tolerance/state.md b/docs/content.zh/docs/dev/datastream/fault-tolerance/state.md index ac260aa429eb5..8e2049054c5e1 100644 --- a/docs/content.zh/docs/dev/datastream/fault-tolerance/state.md +++ b/docs/content.zh/docs/dev/datastream/fault-tolerance/state.md @@ -383,6 +383,8 @@ Heap state backend 会额外存储一个包括用户状态以及时间戳的 Jav - 当前开启 TTL 的 map state 仅在用户值序列化器支持 null 的情况下,才支持用户值为 null。如果用户值序列化器不支持 null, 可以用 `NullableSerializer` 包装一层。 + +- 启用 TTL 配置后,`StateDescriptor` 中的 `defaultValue`(已被标记 `deprecated`)将会失效。这个设计的目的是为了确保语义更加清晰,在此基础上,用户需要手动管理那些实际值为 null 或已过期的状态默认值。 #### 过期数据的清理 diff --git a/docs/content.zh/docs/dev/datastream/operators/joining.md b/docs/content.zh/docs/dev/datastream/operators/joining.md index 32695b703f8da..4da2aa1d3665c 100644 --- a/docs/content.zh/docs/dev/datastream/operators/joining.md +++ b/docs/content.zh/docs/dev/datastream/operators/joining.md @@ -38,7 +38,7 @@ stream.join(otherStream) .where() .equalTo() .window() - .apply() + .apply(); ``` 语义上有一些值得注意的地方: @@ -63,8 +63,8 @@ import org.apache.flink.streaming.api.windowing.time.Time; ... -DataStream orangeStream = ... -DataStream greenStream = ... +DataStream orangeStream = ...; +DataStream greenStream = ...; orangeStream.join(greenStream) .where() @@ -118,8 +118,8 @@ import org.apache.flink.streaming.api.windowing.time.Time; ... -DataStream orangeStream = ... -DataStream greenStream = ... +DataStream orangeStream = ...; +DataStream greenStream = ...; orangeStream.join(greenStream) .where() @@ -171,8 +171,8 @@ import org.apache.flink.streaming.api.windowing.time.Time; ... -DataStream orangeStream = ... -DataStream greenStream = ... +DataStream orangeStream = ...; +DataStream greenStream = ...; orangeStream.join(greenStream) .where() @@ -244,8 +244,8 @@ import org.apache.flink.streaming.api.windowing.time.Time; ... -DataStream orangeStream = ... -DataStream greenStream = ... +DataStream orangeStream = ...; +DataStream greenStream = ...; orangeStream .keyBy() diff --git a/docs/content.zh/docs/dev/datastream/operators/overview.md b/docs/content.zh/docs/dev/datastream/operators/overview.md index 223839bae502e..e0da3f5d91140 100644 --- a/docs/content.zh/docs/dev/datastream/operators/overview.md +++ b/docs/content.zh/docs/dev/datastream/operators/overview.md @@ -680,6 +680,8 @@ data_stream.broadcast() {{< /tab >}} {{< /tabs>}} + + ## 算子链和资源组 将两个算子链接在一起能使得它们在同一个线程中执行,从而提升性能。Flink 默认会将能链接的算子尽可能地进行链接(例如, 两个 map 转换操作)。此外, Flink 还提供了对链接更细粒度控制的 API 以满足更多需求: @@ -769,7 +771,7 @@ Flink里的算子和作业节点会有一个名字和一个描述。名字和描 {{< tabs namedescription>}} {{< tab "Java" >}} ```java -someStream.filter(...).setName("filter").setDescription("x in (1, 2, 3, 4) and y > 1") +someStream.filter(...).setName("filter").setDescription("x in (1, 2, 3, 4) and y > 1"); ``` {{< /tab >}} {{< tab "Scala" >}} diff --git a/docs/content.zh/docs/dev/datastream/operators/process_function.md b/docs/content.zh/docs/dev/datastream/operators/process_function.md index 314d1f33c0a21..0268308fdc18b 100644 --- a/docs/content.zh/docs/dev/datastream/operators/process_function.md +++ b/docs/content.zh/docs/dev/datastream/operators/process_function.md @@ -53,7 +53,7 @@ to apply the `ProcessFunction` on a keyed stream: {{< /hint >}} ```java -stream.keyBy(...).process(new MyProcessFunction()) +stream.keyBy(...).process(new MyProcessFunction()); ``` ## Low-level Joins @@ -461,7 +461,7 @@ Stopping a processing-time timer: {{< tabs "5d0d1344-6f51-44f8-b500-ebe863cedba4" >}} {{< tab "Java" >}} ```java -long timestampOfTimerToStop = ... +long timestampOfTimerToStop = ...; ctx.timerService().deleteProcessingTimeTimer(timestampOfTimerToStop); ``` {{< /tab >}} @@ -484,7 +484,7 @@ Stopping an event-time timer: {{< tabs "581e5996-503c-452e-8b2a-a4daeaf4ac88" >}} {{< tab "Java" >}} ```java -long timestampOfTimerToStop = ... +long timestampOfTimerToStop = ...; ctx.timerService().deleteEventTimeTimer(timestampOfTimerToStop); ``` {{< /tab >}} diff --git a/docs/content.zh/docs/dev/datastream/overview.md b/docs/content.zh/docs/dev/datastream/overview.md index b4e20598dbcc6..164e6227b671c 100644 --- a/docs/content.zh/docs/dev/datastream/overview.md +++ b/docs/content.zh/docs/dev/datastream/overview.md @@ -69,11 +69,11 @@ Flink 程序看起来像一个转换 `DataStream` 的常规程序。每个程序 `StreamExecutionEnvironment` 是所有 Flink 程序的基础。你可以使用 `StreamExecutionEnvironment` 的如下静态方法获取 `StreamExecutionEnvironment`: ```java -getExecutionEnvironment() +getExecutionEnvironment(); -createLocalEnvironment() +createLocalEnvironment(); -createRemoteEnvironment(String host, int port, String... jarFiles) +createRemoteEnvironment(String host, int port, String... jarFiles); ``` 通常,你只需要使用 `getExecutionEnvironment()` 即可,因为该方法会根据上下文做正确的处理:如果你在 IDE 中执行你的程序或将其作为一般的 Java 程序执行,那么它将创建一个本地环境,该环境将在你的本地机器上执行你的程序。如果你基于程序创建了一个 JAR 文件,并通过[命令行]({{< ref "docs/deployment/cli" >}})运行它,Flink 集群管理器将执行程序的 main 方法,同时 `getExecutionEnvironment()` 方法会返回一个执行环境以在集群上执行你的程序。 @@ -106,9 +106,9 @@ DataStream parsed = input.map(new MapFunction() { 一旦你有了包含最终结果的 DataStream,你就可以通过创建 sink 把它写到外部系统。下面是一些用于创建 sink 的示例方法: ```java -writeAsText(String path) +writeAsText(String path); -print() +print(); ``` {{< /tab >}} diff --git a/docs/content.zh/docs/dev/datastream/project-configuration.md b/docs/content.zh/docs/dev/datastream/project-configuration.md deleted file mode 100644 index 24c8bf88939d5..0000000000000 --- a/docs/content.zh/docs/dev/datastream/project-configuration.md +++ /dev/null @@ -1,570 +0,0 @@ ---- -title: "Project Configuration" -weight: 302 -type: docs -aliases: - - /zh/dev/project-configuration.html - - /zh/start/dependencies.html - - /zh/getting-started/project-setup/dependencies.html - - /zh/quickstart/java_api_quickstart.html - - /zh/dev/projectsetup/java_api_quickstart.html - - /zh/dev/linking_with_flink.html - - /zh/dev/linking.html - - /zh/dev/projectsetup/dependencies.html - - /zh/dev/projectsetup/java_api_quickstart.html - - /zh/getting-started/project-setup/java_api_quickstart.html - - /zh/dev/projectsetup/scala_api_quickstart.html - - /zh/getting-started/project-setup/scala_api_quickstart.html - - /zh/quickstart/scala_api_quickstart.html ---- - - -# Project Configuration - -Every Flink application depends on a set of Flink libraries. At the bare minimum, the application depends -on the Flink APIs. Many applications depend in addition on certain connector libraries (like Kafka, Cassandra, etc.). -When running Flink applications (either in a distributed deployment, or in the IDE for testing), the Flink -runtime library must be available as well. - -## Flink Core and Application Dependencies - -As with most systems that run user-defined applications, there are two broad categories of dependencies and libraries in Flink: - - - **Flink Core Dependencies**: Flink itself consists of a set of classes and dependencies that are needed to run the system, for example - coordination, networking, checkpoints, failover, APIs, operations (such as windowing), resource management, etc. - The set of all these classes and dependencies forms the core of Flink's runtime and must be present when a Flink - application is started. - - These core classes and dependencies are packaged in the `flink-dist` jar. They are part of Flink's `lib` folder and - part of the basic Flink container images. Think of these dependencies as similar to Java's core library (`rt.jar`, `charsets.jar`, etc.), - which contains the classes like `String` and `List`. - - The Flink Core Dependencies do not contain any connectors or libraries (CEP, SQL, ML, etc.) in order to avoid having an excessive - number of dependencies and classes in the classpath by default. In fact, we try to keep the core dependencies as slim as possible - to keep the default classpath small and avoid dependency clashes. - - - The **User Application Dependencies** are all connectors, formats, or libraries that a specific user application needs. - - The user application is typically packaged into an *application jar*, which contains the application code and the required - connector and library dependencies. - - The user application dependencies explicitly do not include the Flink DataStream APIs and runtime dependencies, - because those are already part of Flink's Core Dependencies. - - -## Setting up a Project: Basic Dependencies - -Every Flink application needs as the bare minimum the API dependencies, to develop against. - -When setting up a project manually, you need to add the following dependencies for the Java/Scala API -(here presented in Maven syntax, but the same dependencies apply to other build tools (Gradle, SBT, etc.) as well. - -{{< tabs "a49d57a4-27ee-4dd3-a2b8-a673b99b011e" >}} -{{< tab "Java" >}} -```xml - - org.apache.flink - flink-streaming-java - {{< version >}} - provided - -``` -{{< /tab >}} -{{< tab "Scala" >}} -```xml - - org.apache.flink - flink-streaming-scala{{< scala_version >}} - {{< version >}} - provided - -``` -{{< /tab >}} -{{< /tabs >}} - -**Important:** Please note that all these dependencies have their scope set to *provided*. -That means that they are needed to compile against, but that they should not be packaged into the -project's resulting application jar file - these dependencies are Flink Core Dependencies, -which are already available in any setup. - -It is highly recommended keeping the dependencies in scope *provided*. If they are not set to *provided*, -the best case is that the resulting JAR becomes excessively large, because it also contains all Flink core -dependencies. The worst case is that the Flink core dependencies that are added to the application's jar file -clash with some of your own dependency versions (which is normally avoided through inverted classloading). - -**Note on IntelliJ:** To make the applications run within IntelliJ IDEA it is necessary to tick the -`Include dependencies with "Provided" scope` box in the run configuration. -If this option is not available (possibly due to using an older IntelliJ IDEA version), then a simple workaround -is to create a test that calls the applications `main()` method. - - -## Adding Connector and Library Dependencies - -Most applications need specific connectors or libraries to run, for example a connector to Kafka, Cassandra, etc. -These connectors are not part of Flink's core dependencies and must be added as dependencies to the application. - -Below is an example adding the connector for Kafka as a dependency (Maven syntax): -```xml - - org.apache.flink - flink-connector-kafka - {{< version >}} - -``` - -We recommend packaging the application code and all its required dependencies into one *jar-with-dependencies* which -we refer to as the *application jar*. The application jar can be submitted to an already running Flink cluster, -or added to a Flink application container image. - -Projects created from the [Java Project Template]({{< ref "docs/dev/datastream/project-configuration" >}}) or -[Scala Project Template]({{< ref "docs/dev/datastream/project-configuration" >}}) are configured to automatically include -the application dependencies into the application jar when running `mvn clean package`. For projects that are -not set up from those templates, we recommend adding the Maven Shade Plugin (as listed in the Appendix below) -to build the application jar with all required dependencies. - -**Important:** For Maven (and other build tools) to correctly package the dependencies into the application jar, -these application dependencies must be specified in scope *compile* (unlike the core dependencies, which -must be specified in scope *provided*). - - -## Scala Versions - -Scala versions (2.11, 2.12, etc.) are not binary compatible with one another. -For that reason, Flink for Scala 2.11 cannot be used with an application that uses -Scala 2.12. - -All Flink dependencies that (transitively) depend on Scala are suffixed with the -Scala version that they are built for, for example `flink-streaming-scala_2.12`. - -Developers that only use Java can pick any Scala version, Scala developers need to -pick the Scala version that matches their application's Scala version. - -Please refer to the [build guide]({{< ref "docs/flinkDev/building" >}}#scala-versions) -for details on how to build Flink for a specific Scala version. - -Scala versions after 2.12.8 are not binary compatible with previous 2.12.x -versions, preventing the Flink project from upgrading its 2.12.x builds beyond -2.12.8. Users can build Flink locally for latter Scala versions by following -the above mentioned [build guide]({{< ref "docs/flinkDev/building" >}}#scala-versions). -For this to work, users need to add `-Djapicmp.skip` to -skip binary compatibility checks when building. - -See the [Scala 2.12.8 release notes](https://github.com/scala/scala/releases/tag/v2.12.8) for more details, -the relevant quote is this: - -> The second fix is not binary compatible: the 2.12.8 compiler omits certain -> methods that are generated by earlier 2.12 compilers. However, we believe -> that these methods are never used and existing compiled code will continue to -> work. See the [pull request -> description](https://github.com/scala/scala/pull/7469) for more details. - -## Hadoop Dependencies - -**General rule: It should never be necessary to add Hadoop dependencies directly to your application.** -*(The only exception being when using existing Hadoop input-/output formats with Flink's Hadoop compatibility wrappers)* - -If you want to use Flink with Hadoop, you need to have a Flink setup that includes the Hadoop dependencies, rather than -adding Hadoop as an application dependency. Flink will use the Hadoop dependencies specified by the `HADOOP_CLASSPATH` -environment variable, which can be set in the following way: - -```bash -export HADOOP_CLASSPATH=`hadoop classpath` -``` - -There are two main reasons for that design: - - - Some Hadoop interaction happens in Flink's core, possibly before the user application is started, for example - setting up HDFS for checkpoints, authenticating via Hadoop's Kerberos tokens, or deployment on YARN. - - - Flink's inverted classloading approach hides many transitive dependencies from the core dependencies. That applies not only - to Flink's own core dependencies, but also to Hadoop's dependencies when present in the setup. - That way, applications can use different versions of the same dependencies without running into dependency conflicts (and - trust us, that's a big deal, because Hadoops dependency tree is huge.) - -If you need Hadoop dependencies during testing or development inside the IDE (for example for HDFS access), please configure -these dependencies similar to the scope of the dependencies to *test* or to *provided*. - -## Maven Quickstart - -#### Requirements - -The only requirements are working __Maven 3.0.4__ (or higher) and __Java 11__ installations. - -#### Create Project - -Use one of the following commands to __create a project__: - -{{< tabs "maven" >}} -{{< tab "Maven Archetypes" >}} -```bash -$ mvn archetype:generate \ - -DarchetypeGroupId=org.apache.flink \ - -DarchetypeArtifactId=flink-quickstart-java \ - -DarchetypeVersion={{< version >}} -``` -This allows you to **name your newly created project**. -It will interactively ask you for the groupId, artifactId, and package name. -{{< /tab >}} -{{< tab "Quickstart Script" >}} -{{< stable >}} -```bash -$ curl https://flink.apache.org/q/quickstart.sh | bash -s {{< version >}} -``` -{{< /stable >}} -{{< unstable >}} -```bash -$ curl https://flink.apache.org/q/quickstart-SNAPSHOT.sh | bash -s {{< version >}} - -``` -{{< /unstable >}} -{{< /tab >}} -{{< /tabs >}} - -{{< unstable >}} -{{< hint info >}} -For Maven 3.0 or higher, it is no longer possible to specify the repository (-DarchetypeCatalog) via the command line. For details about this change, please refer to Maven official document -If you wish to use the snapshot repository, you need to add a repository entry to your settings.xml. For example: - -```xml - - - apache - - - - apache - - - apache-snapshots - https://repository.apache.org/content/repositories/snapshots/ - - - - - -``` - -{{< /hint >}} -{{< /unstable >}} - -We recommend you __import this project into your IDE__ to develop and -test it. IntelliJ IDEA supports Maven projects out of the box. -If you use Eclipse, the [m2e plugin](http://www.eclipse.org/m2e/) -allows to [import Maven projects](http://books.sonatype.com/m2eclipse-book/reference/creating-sect-importing-projects.html#fig-creating-import). -Some Eclipse bundles include that plugin by default, others require you -to install it manually. - -*Please note*: The default JVM heapsize for Java may be too -small for Flink. You have to manually increase it. -In Eclipse, choose `Run Configurations -> Arguments` and write into the `VM Arguments` box: `-Xmx800m`. -In IntelliJ IDEA recommended way to change JVM options is from the `Help | Edit Custom VM Options` menu. See [this article](https://intellij-support.jetbrains.com/hc/en-us/articles/206544869-Configuring-JVM-options-and-platform-properties) for details. - - -#### Build Project - -If you want to __build/package your project__, go to your project directory and -run the '`mvn clean package`' command. -You will __find a JAR file__ that contains your application, plus connectors and libraries -that you may have added as dependencies to the application: `target/-.jar`. - -__Note:__ If you use a different class than *StreamingJob* as the application's main class / entry point, -we recommend you change the `mainClass` setting in the `pom.xml` file accordingly. That way, Flink -can run the application from the JAR file without additionally specifying the main class. - -## Gradle - -#### Requirements - -The only requirements are working __Gradle 3.x__ (or higher) and __Java 11__ installations. - -#### Create Project - -Use one of the following commands to __create a project__: - -{{< tabs gradle >}} -{{< tab "Gradle Example" >}} -**build.gradle** - -```gradle -buildscript { - repositories { - jcenter() // this applies only to the Gradle 'Shadow' plugin - } - dependencies { - classpath 'com.github.jengelman.gradle.plugins:shadow:2.0.4' - } -} - -plugins { - id 'java' - id 'application' - // shadow plugin to produce fat JARs - id 'com.github.johnrengelman.shadow' version '2.0.4' -} - - -// artifact properties -group = 'org.myorg.quickstart' -version = '0.1-SNAPSHOT' -mainClassName = 'org.myorg.quickstart.StreamingJob' -description = """Flink Quickstart Job""" - -ext { - javaVersion = '1.8' - flinkVersion = '1.13-SNAPSHOT' - scalaBinaryVersion = '2.11' - slf4jVersion = '1.7.32' - log4jVersion = '2.17.1' -} - - -sourceCompatibility = javaVersion -targetCompatibility = javaVersion -tasks.withType(JavaCompile) { - options.encoding = 'UTF-8' -} - -applicationDefaultJvmArgs = ["-Dlog4j.configurationFile=log4j2.properties"] - -task wrapper(type: Wrapper) { - gradleVersion = '3.1' -} - -// declare where to find the dependencies of your project -repositories { - mavenCentral() - maven { url "https://repository.apache.org/content/repositories/snapshots/" } -} - -// NOTE: We cannot use "compileOnly" or "shadow" configurations since then we could not run code -// in the IDE or with "gradle run". We also cannot exclude transitive dependencies from the -// shadowJar yet (see https://github.com/johnrengelman/shadow/issues/159). -// -> Explicitly define the // libraries we want to be included in the "flinkShadowJar" configuration! -configurations { - flinkShadowJar // dependencies which go into the shadowJar - - // always exclude these (also from transitive dependencies) since they are provided by Flink - flinkShadowJar.exclude group: 'org.apache.flink', module: 'force-shading' - flinkShadowJar.exclude group: 'com.google.code.findbugs', module: 'jsr305' - flinkShadowJar.exclude group: 'org.slf4j' - flinkShadowJar.exclude group: 'org.apache.logging.log4j' -} - -// declare the dependencies for your production and test code -dependencies { - // -------------------------------------------------------------- - // Compile-time dependencies that should NOT be part of the - // shadow jar and are provided in the lib folder of Flink - // -------------------------------------------------------------- - compile "org.apache.flink:flink-streaming-java:${flinkVersion}" - - // -------------------------------------------------------------- - // Dependencies that should be part of the shadow jar, e.g. - // connectors. These must be in the flinkShadowJar configuration! - // -------------------------------------------------------------- - //flinkShadowJar "org.apache.flink:flink-connector-kafka:${flinkVersion}" - - compile "org.apache.logging.log4j:log4j-api:${log4jVersion}" - compile "org.apache.logging.log4j:log4j-core:${log4jVersion}" - compile "org.apache.logging.log4j:log4j-slf4j-impl:${log4jVersion}" - compile "org.slf4j:slf4j-log4j12:${slf4jVersion}" - - // Add test dependencies here. - // testCompile "junit:junit:4.12" -} - -// make compileOnly dependencies available for tests: -sourceSets { - main.compileClasspath += configurations.flinkShadowJar - main.runtimeClasspath += configurations.flinkShadowJar - - test.compileClasspath += configurations.flinkShadowJar - test.runtimeClasspath += configurations.flinkShadowJar - - javadoc.classpath += configurations.flinkShadowJar -} - -run.classpath = sourceSets.main.runtimeClasspath - -jar { - manifest { - attributes 'Built-By': System.getProperty('user.name'), - 'Build-Jdk': System.getProperty('java.version') - } -} - -shadowJar { - configurations = [project.configurations.flinkShadowJar] -} -``` - -**settings.gradle** -```gradle -rootProject.name = 'quickstart' -``` -{{< /tab >}} -{{< tab "Quickstart Script">}} -```bash -bash -c "$(curl https://flink.apache.org/q/gradle-quickstart.sh)" -- {{< version >}} {{< scala_version >}} -``` -{{< /tab >}} -{{< /tabs >}} - -We recommend you __import this project into your IDE__ to develop and -test it. IntelliJ IDEA supports Gradle projects after installing the `Gradle` plugin. -Eclipse does so via the [Eclipse Buildship](https://projects.eclipse.org/projects/tools.buildship) plugin -(make sure to specify a Gradle version >= 3.0 in the last step of the import wizard; the `shadow` plugin requires it). -You may also use [Gradle's IDE integration](https://docs.gradle.org/current/userguide/userguide.html#ide-integration) -to create project files from Gradle. - - -*Please note*: The default JVM heapsize for Java may be too -small for Flink. You have to manually increase it. -In Eclipse, choose `Run Configurations -> Arguments` and write into the `VM Arguments` box: `-Xmx800m`. -In IntelliJ IDEA recommended way to change JVM options is from the `Help | Edit Custom VM Options` menu. See [this article](https://intellij-support.jetbrains.com/hc/en-us/articles/206544869-Configuring-JVM-options-and-platform-properties) for details. - -#### Build Project - -If you want to __build/package your project__, go to your project directory and -run the '`gradle clean shadowJar`' command. -You will __find a JAR file__ that contains your application, plus connectors and libraries -that you may have added as dependencies to the application: `build/libs/--all.jar`. - -__Note:__ If you use a different class than *DataStreamJob* as the application's main class / entry point, -we recommend you change the `mainClassName` setting in the `build.gradle` file accordingly. That way, Flink -can run the application from the JAR file without additionally specifying the main class. - -## SBT - -#### Create Project - -You can scaffold a new project via either of the following two methods: - -{{< tabs sbt >}} -{{< tab "SBT Template" >}} -```bash -$ sbt new tillrohrmann/flink-project.g8 -``` -{{< /tab >}} -{{< tab "Quickstart Script" >}} -```bash -$ bash <(curl https://flink.apache.org/q/sbt-quickstart.sh) -``` -{{< /tab >}} -{{< /tabs >}} - -#### Build Project - -In order to build your project you simply have to issue the `sbt clean assembly` command. -This will create the fat-jar __your-project-name-assembly-0.1-SNAPSHOT.jar__ in the directory __target/scala_your-major-scala-version/__. - -#### Run Project - -In order to run your project you have to issue the `sbt run` command. - -Per default, this will run your job in the same JVM as `sbt` is running. -In order to run your job in a distinct JVM, add the following line to `build.sbt` - -```scala -fork in run := true -``` - -#### IntelliJ - -We recommend using [IntelliJ](https://www.jetbrains.com/idea/) for your Flink job development. -In order to get started, you have to import your newly created project into IntelliJ. -You can do this via `File -> New -> Project from Existing Sources...` and then choosing your project's directory. -IntelliJ will then automatically detect the `build.sbt` file and set everything up. - -In order to run your Flink job, it is recommended to choose the `mainRunner` module as the classpath of your __Run/Debug Configuration__. -This will ensure, that all dependencies which are set to _provided_ will be available upon execution. -You can configure the __Run/Debug Configurations__ via `Run -> Edit Configurations...` and then choose `mainRunner` from the _Use classpath of module_ dropbox. - -#### Eclipse - -In order to import the newly created project into [Eclipse](https://eclipse.org/), you first have to create Eclipse project files for it. -These project files can be created via the [sbteclipse](https://github.com/typesafehub/sbteclipse) plugin. -Add the following line to your `PROJECT_DIR/project/plugins.sbt` file: - -```bash -addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "4.0.0") -``` - -In `sbt` use the following command to create the Eclipse project files - -```bash -> eclipse -``` - -Now you can import the project into Eclipse via `File -> Import... -> Existing Projects into Workspace` and then select the project directory. - - -## Appendix: Template for building a Jar with Dependencies - -To build an application JAR that contains all dependencies required for declared connectors and libraries, -you can use the following shade plugin definition: - -```xml - - - - org.apache.maven.plugins - maven-shade-plugin - 3.1.1 - - - package - - shade - - - - - com.google.code.findbugs:jsr305 - org.slf4j:* - log4j:* - - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - my.programs.main.clazz - - - - - - - - -``` - -{{< top >}} diff --git a/docs/content.zh/docs/dev/datastream/sources.md b/docs/content.zh/docs/dev/datastream/sources.md index 0c3b7fe35aa64..b01c631fe010e 100644 --- a/docs/content.zh/docs/dev/datastream/sources.md +++ b/docs/content.zh/docs/dev/datastream/sources.md @@ -363,7 +363,7 @@ Source 的实现需要完成一部分*事件时间*分配和*水印生成*的工 environment.fromSource( Source source, WatermarkStrategy timestampsAndWatermarks, - String sourceName) + String sourceName); ``` `TimestampAssigner` 和 `WatermarkGenerator` 作为 `ReaderOutput`(或 `SourceOutput`)的一部分透明地运行,因此 Source 实现者不必实现任何时间戳提取和水印生成的代码。 diff --git a/docs/content.zh/docs/dev/datastream/testing.md b/docs/content.zh/docs/dev/datastream/testing.md index c8e709366665e..5fdd52c746c3d 100644 --- a/docs/content.zh/docs/dev/datastream/testing.md +++ b/docs/content.zh/docs/dev/datastream/testing.md @@ -151,11 +151,7 @@ class IncrementFlatMapFunctionTest extends FlatSpec with MockFactory { * `TwoInputStreamOperatorTestHarness` (f适用于两个 `DataStream` 的 `ConnectedStreams` 算子) * `KeyedTwoInputStreamOperatorTestHarness` (适用于两个 `KeyedStream` 上的 `ConnectedStreams` 算子) -要使用测试工具,还需要一组其他的依赖项(测试范围)。 - -{{< artifact flink-test-utils withTestScope >}} -{{< artifact flink-runtime withTestScope >}} -{{< artifact flink-streaming-java withTestScope withTestClassifier >}} +要使用测试工具,还需要一组其他的依赖项,请查阅[配置]({{< ref "docs/dev/configuration/testing" >}})小节了解更多细节。 现在,可以使用测试工具将记录和 watermark 推送到用户自定义函数或自定义算子中,控制处理时间,最后对算子的输出(包括旁路输出)进行校验。 diff --git a/docs/content.zh/docs/dev/datastream/user_defined_functions.md b/docs/content.zh/docs/dev/datastream/user_defined_functions.md index c8c0f3b2b18dd..652fb5a6d2966 100644 --- a/docs/content.zh/docs/dev/datastream/user_defined_functions.md +++ b/docs/content.zh/docs/dev/datastream/user_defined_functions.md @@ -209,7 +209,7 @@ this.numLines.add(1); 最终整体结果会存储在由执行环境的 `execute()` 方法返回的 ```JobExecutionResult``` 对象中(当前只有等待作业完成后执行才起作用)。 ```java -myJobExecutionResult.getAccumulatorResult("num-lines") +myJobExecutionResult.getAccumulatorResult("num-lines"); ``` 单个作业的所有累加器共享一个命名空间。因此你可以在不同的操作 function 里面使用同一个累加器。Flink 会在内部将所有具有相同名称的累加器合并起来。 diff --git a/docs/content.zh/docs/dev/python/datastream/intro_to_datastream_api.md b/docs/content.zh/docs/dev/python/datastream/intro_to_datastream_api.md index 4605c8b750b58..5674e6e0fb6a0 100644 --- a/docs/content.zh/docs/dev/python/datastream/intro_to_datastream_api.md +++ b/docs/content.zh/docs/dev/python/datastream/intro_to_datastream_api.md @@ -78,7 +78,7 @@ def state_access_demo(): # 3. define the execution logic ds = ds.map(lambda a: Row(a % 4, 1), output_type=Types.ROW([Types.LONG(), Types.LONG()])) \ .key_by(lambda a: a[0]) \ - .map(MyMapFunction(), output_type=Types.ROW([Types.LONG(), Types.LONG()])) + .map(MyMapFunction(), output_type=Types.TUPLE([Types.LONG(), Types.LONG()])) # 4. create sink and emit result to sink output_path = '/opt/output/' diff --git a/docs/content.zh/docs/dev/python/debugging.md b/docs/content.zh/docs/dev/python/debugging.md index 2d54424acee47..0cefc596b59a8 100644 --- a/docs/content.zh/docs/dev/python/debugging.md +++ b/docs/content.zh/docs/dev/python/debugging.md @@ -116,7 +116,7 @@ $ python -c "import pyflink;import os;print(os.path.dirname(os.path.abspath(pyfl 你可以打开profile来分析性能瓶颈 ```python -t_env.get_config().get_configuration().set_boolean("python.profile.enabled", True) +t_env.get_config().set("python.profile.enabled", "true") ``` 你可以在[日志](#查看日志)里面查看profile的结果 diff --git a/docs/content.zh/docs/dev/python/dependency_management.md b/docs/content.zh/docs/dev/python/dependency_management.md index c8186865136b5..52b584548a8ab 100644 --- a/docs/content.zh/docs/dev/python/dependency_management.md +++ b/docs/content.zh/docs/dev/python/dependency_management.md @@ -51,12 +51,15 @@ If third-party JARs are used, you can specify the JARs in the Python Table API a # Specify a list of jar URLs via "pipeline.jars". The jars are separated by ";" # and will be uploaded to the cluster. # NOTE: Only local file URLs (start with "file://") are supported. -table_env.get_config().get_configuration().set_string("pipeline.jars", "file:///my/jar/path/connector.jar;file:///my/jar/path/udf.jar") +table_env.get_config().set("pipeline.jars", "file:///my/jar/path/connector.jar;file:///my/jar/path/udf.jar") + +# It looks like the following on Windows: +table_env.get_config().set("pipeline.jars", "file:///E:/my/jar/path/connector.jar;file:///E:/my/jar/path/udf.jar") # Specify a list of URLs via "pipeline.classpaths". The URLs are separated by ";" # and will be added to the classpath during job execution. # NOTE: The paths must specify a protocol (e.g. file://) and users should ensure that the URLs are accessible on both the client and the cluster. -table_env.get_config().get_configuration().set_string("pipeline.classpaths", "file:///my/jar/path/connector.jar;file:///my/jar/path/udf.jar") +table_env.get_config().set("pipeline.classpaths", "file:///my/jar/path/connector.jar;file:///my/jar/path/udf.jar") ``` or in the Python DataStream API as following: @@ -66,6 +69,9 @@ or in the Python DataStream API as following: # NOTE: Only local file URLs (start with "file://") are supported. stream_execution_environment.add_jars("file:///my/jar/path/connector1.jar", "file:///my/jar/path/connector2.jar") +# It looks like the following on Windows: +stream_execution_environment.add_jars("file:///E:/my/jar/path/connector1.jar", "file:///E:/my/jar/path/connector2.jar") + # Use the add_classpaths() to add the dependent jars URLs into the classpath. # The URLs will also be added to the classpath of both the client and the cluster. # NOTE: The paths must specify a protocol (e.g. file://) and users should ensure that the diff --git a/docs/content.zh/docs/dev/python/faq.md b/docs/content.zh/docs/dev/python/faq.md index 54faf6b9a15c2..939f6b4a441b6 100644 --- a/docs/content.zh/docs/dev/python/faq.md +++ b/docs/content.zh/docs/dev/python/faq.md @@ -64,10 +64,10 @@ PyFlink作业可能依赖jar文件,比如connector,Java UDF等。 ```python # 注意:仅支持本地文件URL(以"file:"开头)。 -table_env.get_config().get_configuration().set_string("pipeline.jars", "file:///my/jar/path/connector.jar;file:///my/jar/path/udf.jar") +table_env.get_config().set("pipeline.jars", "file:///my/jar/path/connector.jar;file:///my/jar/path/udf.jar") # 注意:路径必须指定协议(例如:文件——"file"),并且用户应确保在客户端和群集上都可以访问这些URL。 -table_env.get_config().get_configuration().set_string("pipeline.classpaths", "file:///my/jar/path/connector.jar;file:///my/jar/path/udf.jar") +table_env.get_config().set("pipeline.classpaths", "file:///my/jar/path/connector.jar;file:///my/jar/path/udf.jar") ``` 有关添加Java依赖项的API的详细信息,请参阅[相关文档]({{< ref "docs/dev/python/dependency_management" >}}#java-dependency-in-python-program)。 diff --git a/docs/content.zh/docs/dev/python/python_config.md b/docs/content.zh/docs/dev/python/python_config.md index 664fd8594cb8f..8c72def7a5195 100644 --- a/docs/content.zh/docs/dev/python/python_config.md +++ b/docs/content.zh/docs/dev/python/python_config.md @@ -49,9 +49,23 @@ from pyflink.table import TableEnvironment, EnvironmentSettings env_settings = EnvironmentSettings.in_streaming_mode() t_env = TableEnvironment.create(env_settings) +t_env.get_config().set("python.fn-execution.bundle.size", "1000") +``` -config = t_env.get_config().get_configuration() -config.set_integer("python.fn-execution.bundle.size", 1000) +The config options could also be set when creating EnvironmentSettings: +```python +from pyflink.common import Configuration +from pyflink.table import TableEnvironment, EnvironmentSettings + +# create a streaming TableEnvironment +config = Configuration() +config.set_string("python.fn-execution.bundle.size", "1000") +env_settings = EnvironmentSettings \ + .new_instance() \ + .in_streaming_mode() \ + .with_configuration(config) \ + .build() +table_env = TableEnvironment.create(env_settings) ``` ## Python Options diff --git a/docs/content.zh/docs/dev/python/python_execution_mode.md b/docs/content.zh/docs/dev/python/python_execution_mode.md index c34da240a20ba..dae060ec65c0a 100644 --- a/docs/content.zh/docs/dev/python/python_execution_mode.md +++ b/docs/content.zh/docs/dev/python/python_execution_mode.md @@ -31,61 +31,48 @@ defines how to execute your customized Python functions. Prior to release-1.15, there is the only execution mode called `PROCESS` execution mode. The `PROCESS` mode means that the Python user-defined functions will be executed in separate Python processes. -In release-1.15, it has introduced another two execution modes called `MULTI-THREAD` execution mode and -`SUB-INTERPRETER` execution mode. The `MULTI-THREAD` mode means that the Python user-defined functions -will be executed in the same thread as Java Operator, but it will be affected by GIL performance. -The `SUB-INTERPRETER` mode means that the Python user-defined functions will be executed in Python -different sub-interpreters rather than different threads of one interpreter, which can largely overcome -the effects of the GIL, but some CPython extensions libraries doesn't support it, such as numpy, tensorflow, etc. +In release-1.15, it has introduced a new execution mode called `THREAD` execution mode. The `THREAD` +mode means that the Python user-defined functions will be executed in the same process as Java Operator, +It should be noted that multiple Python user-defined functions running in the same JVM are still affected by GIL. -## When can/should I use MULTI-THREAD execution mode or SUB-INTERPRETER execution mode? +## When can/should I use THREAD execution mode? -The purpose of the introduction of `MULTI-THREAD` mode and `SUB-INTERPRETER` mode is to overcome the -overhead of serialization/deserialization and network communication caused in `PROCESS` mode. -So if performance is not your concern, or the computing logic of your customized Python functions is -the performance bottleneck of the job, `PROCESS` mode will be the best choice as `PROCESS` mode provides -the best isolation compared to `MULTI-THREAD` mode and `SUB-INTERPRETER` mode. - -Compared to `MULTI-THREAD` execution mode, `SUB-INTERPRETER` execution mode can largely overcome the -effects of the GIL, so you can get better performance usually. However, `SUB-INTERPRETER` may fail in some CPython -extensions libraries, such as numpy, tensorflow. In this case, you should use `PROCESS` mode or `MULTI-THREAD` mode. +The purpose of the introduction of `THREAD` mode is to overcome the overhead of serialization/deserialization +and network communication caused in `PROCESS` mode. So if performance is not your concern, or the computing +logic of your customized Python functions is the performance bottleneck of the job, `PROCESS` mode will +be the best choice as `PROCESS` mode provides the best isolation compared to `THREAD` mode. ## Configuring Python execution mode The execution mode can be configured via the `python.execution-mode` setting. -There are three possible values: +There are two possible values: - `PROCESS`: The Python user-defined functions will be executed in separate Python process. (default) - - `MULTI-THREAD`: The Python user-defined functions will be executed in the same thread as Java Operator. - - `SUB-INTERPRETER`: The Python user-defined functions will be executed in Python different sub-interpreters. + - `THREAD`: The Python user-defined functions will be executed in the same process as Java operator. You could specify the Python execution mode using Python Table API as following: ```python # Specify `PROCESS` mode -table_env.get_config().get_configuration().set_string("python.execution-mode", "process") - -# Specify `MULTI-THREAD` mode -table_env.get_config().get_configuration().set_string("python.execution-mode", "multi-thread") +table_env.get_config().set("python.execution-mode", "process") -# Specify `SUB-INTERPRETER` mode -table_env.get_config().get_configuration().set_string("python.execution-mode", "sub-interpreter") +# Specify `THREAD` mode +table_env.get_config().set("python.execution-mode", "thread") ``` {{< hint info >}} -Currently, it still doesn't support to execute Python UDFs in `MULTI-THREAD` and `SUB-INTERPRETER` execution mode -in all places. It will fall back to `PROCESS` execution mode in these cases. So it may happen that you configure a job -to execute in `MULTI-THREAD` or `SUB-INTERPRETER` execution modes, however, it's actually executed in `PROCESS` execution mode. +Currently, it still doesn't support to execute Python UDFs in `THREAD` execution mode in all places. +It will fall back to `PROCESS` execution mode in these cases. So it may happen that you configure a job +to execute in `THREAD` execution mode, however, it's actually executed in `PROCESS` execution mode. {{< /hint >}} {{< hint info >}} -`MULTI-THREAD` execution mode only supports Python 3.7+. `SUB-INTERPRETER` execution mode only supports Python 3.8+. +`THREAD` execution mode is only supported in Python 3.7+. {{< /hint >}} ## Execution Behavior -This section provides an overview of the execution behavior of `MULTI-THREAD` and `SUB-INTERPRETER` -execution mode and contrasts they with `PROCESS` execution mode. For more -details, please refer to the FLIP that introduced this feature: +This section provides an overview of the execution behavior of `THREAD` execution mode and contrasts +they with `PROCESS` execution mode. For more details, please refer to the FLIP that introduced this feature: [FLIP-206](https://cwiki.apache.org/confluence/display/FLINK/FLIP-206%3A+Support+PyFlink+Runtime+Execution+in+Thread+Mode). #### PROCESS Execution Mode @@ -95,10 +82,10 @@ The Java operator process communicates with the Python worker process using vari {{< img src="/fig/pyflink_process_execution_mode.png" alt="Process Execution Mode" >}} -#### MULTI-THREAD and SUB-INTERPRETER Execution Mode +#### THREAD Execution Mode -In `MULTI-THREAD` and `SUB-INTERPRETER` execution mode, the Python user-defined functions will be executed in -the same process as Java operators. PyFlink takes use of third part library [PEMJA](https://github.com/alibaba/pemja) to -embed Python in Java Application. +In `THREAD` execution mode, the Python user-defined functions will be executed in the same process +as Java operators. PyFlink takes use of third part library [PEMJA](https://github.com/alibaba/pemja) +to embed Python in Java Application. {{< img src="/fig/pyflink_embedded_execution_mode.png" alt="Embedded Execution Mode" >}} 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 fa3d4a734ab09..6e36a879b59de 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 @@ -29,9 +29,7 @@ under the License. 本文档是对 PyFlink Table API 的简要介绍,用于帮助新手用户快速理解 PyFlink Table API 的基本用法。 关于高级用法,请参阅用户指南中的其他文档。 - - -Python Table API 程序的基本结构 +Python Table API 程序的基本结构 -------------------------------------------- 所有的 Table API 和 SQL 程序,不管批模式,还是流模式,都遵循相同的结构。下面代码示例展示了 Table API 和 SQL 程序的基本结构。 diff --git a/docs/content.zh/docs/dev/python/table/python_table_api_connectors.md b/docs/content.zh/docs/dev/python/table/python_table_api_connectors.md index 8f786b5fd15ad..dbf94fbd1eee9 100644 --- a/docs/content.zh/docs/dev/python/table/python_table_api_connectors.md +++ b/docs/content.zh/docs/dev/python/table/python_table_api_connectors.md @@ -36,7 +36,7 @@ under the License. 要在 PyFlink 作业中使用,首先需要将其指定为作业的 [依赖]({{< ref "docs/dev/python/dependency_management" >}})。 ```python -table_env.get_config().get_configuration().set_string("pipeline.jars", "file:///my/jar/path/connector.jar;file:///my/jar/path/json.jar") +table_env.get_config().set("pipeline.jars", "file:///my/jar/path/connector.jar;file:///my/jar/path/json.jar") ``` ## 如何使用连接器 @@ -86,7 +86,7 @@ def log_processing(): env_settings = EnvironmentSettings.in_streaming_mode() t_env = TableEnvironment.create(env_settings) # specify connector and format jars - t_env.get_config().get_configuration().set_string("pipeline.jars", "file:///my/jar/path/connector.jar;file:///my/jar/path/json.jar") + t_env.get_config().set("pipeline.jars", "file:///my/jar/path/connector.jar;file:///my/jar/path/json.jar") source_ddl = """ CREATE TABLE source_table( 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 bf65cdf050eae..641540b20c1bc 100644 --- a/docs/content.zh/docs/dev/python/table/table_environment.md +++ b/docs/content.zh/docs/dev/python/table/table_environment.md @@ -35,12 +35,18 @@ under the License. 创建 `TableEnvironment` 的推荐方式是通过 `EnvironmentSettings` 对象创建: ```python +from pyflink.common import Configuration from pyflink.table import EnvironmentSettings, TableEnvironment # create a streaming TableEnvironment -env_settings = EnvironmentSettings.in_streaming_mode() -# or a batch TableEnvironment -# env_settings = EnvironmentSettings.in_batch_mode() +config = Configuration() +config.set_string('execution.buffer-timeout', '1 min') +env_settings = EnvironmentSettings \ + .new_instance() \ + .in_streaming_mode() \ + .with_configuration(config) \ + .build() + table_env = TableEnvironment.create(env_settings) ``` @@ -562,12 +568,9 @@ TableEnvironment API 返回 table config,可以通过 table config 来定义 Table API 的运行时行为。 你可以在 }}">配置}}">Python 配置 中找到所有可用的配置选项。

- 下面的代码示例展示了如何通过这个 API 来设置配置选项: -```python -# set the parallelism to 8 -table_env.get_config().get_configuration().set_string( - "parallelism.default", "8") -``` + 下面的代码示例展示了如何通过这个 API 来设置配置选项:
+# set the parallelism to 8
+table_env.get_config().set("parallelism.default", "8") {{< pythondoc file="pyflink.table.html#pyflink.table.TableEnvironment.get_config" name="链接">}} @@ -816,19 +819,19 @@ Statebackend,Checkpoint 以及重启策略 下面代码示例展示了如何通过 Table API 来配置 statebackend,checkpoint 以及重启策略: ```python # 设置重启策略为 "fixed-delay" -table_env.get_config().get_configuration().set_string("restart-strategy", "fixed-delay") -table_env.get_config().get_configuration().set_string("restart-strategy.fixed-delay.attempts", "3") -table_env.get_config().get_configuration().set_string("restart-strategy.fixed-delay.delay", "30s") +table_env.get_config().set("restart-strategy", "fixed-delay") +table_env.get_config().set("restart-strategy.fixed-delay.attempts", "3") +table_env.get_config().set("restart-strategy.fixed-delay.delay", "30s") # 设置 checkpoint 模式为 EXACTLY_ONCE -table_env.get_config().get_configuration().set_string("execution.checkpointing.mode", "EXACTLY_ONCE") -table_env.get_config().get_configuration().set_string("execution.checkpointing.interval", "3min") +table_env.get_config().set("execution.checkpointing.mode", "EXACTLY_ONCE") +table_env.get_config().set("execution.checkpointing.interval", "3min") # 设置 statebackend 类型为 "rocksdb",其他可选项有 "filesystem" 和 "jobmanager" # 你也可以将这个属性设置为 StateBackendFactory 的完整类名 # e.g. org.apache.flink.contrib.streaming.state.RocksDBStateBackendFactory -table_env.get_config().get_configuration().set_string("state.backend", "rocksdb") +table_env.get_config().set("state.backend", "rocksdb") # 设置 RocksDB statebackend 所需要的 checkpoint 目录 -table_env.get_config().get_configuration().set_string("state.checkpoints.dir", "file:///tmp/checkpoints/") +table_env.get_config().set("state.checkpoints.dir", "file:///tmp/checkpoints/") ``` diff --git a/docs/content.zh/docs/dev/python/table_api_tutorial.md b/docs/content.zh/docs/dev/python/table_api_tutorial.md index 7b1f5db8cfbf7..dfe9c78e341a6 100644 --- a/docs/content.zh/docs/dev/python/table_api_tutorial.md +++ b/docs/content.zh/docs/dev/python/table_api_tutorial.md @@ -66,7 +66,7 @@ $ python -m pip install apache-flink ```python t_env = TableEnvironment.create(EnvironmentSettings.in_streaming_mode()) -t_env.get_config().get_configuration().set_string("parallelism.default", "1") +t_env.get_config().set("parallelism.default", "1") ``` 接下来,我们将介绍如何创建源表和结果表。 @@ -200,7 +200,7 @@ word_count_data = ["To be, or not to be,--that is the question:--", def word_count(input_path, output_path): t_env = TableEnvironment.create(EnvironmentSettings.in_streaming_mode()) # write all the data to one file - t_env.get_config().get_configuration().set_string("parallelism.default", "1") + t_env.get_config().set("parallelism.default", "1") # define the source if input_path is not None: diff --git a/docs/content.zh/docs/dev/table/catalogs.md b/docs/content.zh/docs/dev/table/catalogs.md index f12f80553e72c..0b86fbd6769a2 100644 --- a/docs/content.zh/docs/dev/table/catalogs.md +++ b/docs/content.zh/docs/dev/table/catalogs.md @@ -70,7 +70,7 @@ Catalog 是可扩展的,用户可以通过实现 `Catalog` 接口来开发自 {{< tabs "88ed733a-cf54-4676-9685-7d77d3cc9771" >}} {{< tab "Java" >}} ```java -TableEnvironment tableEnv = ... +TableEnvironment tableEnv = ...; // Create a HiveCatalog Catalog catalog = new HiveCatalog("myhive", null, ""); diff --git a/docs/content.zh/docs/dev/table/common.md b/docs/content.zh/docs/dev/table/common.md index a1d5539a07195..02f06056a7fb8 100644 --- a/docs/content.zh/docs/dev/table/common.md +++ b/docs/content.zh/docs/dev/table/common.md @@ -53,7 +53,7 @@ tableEnv.createTemporaryTable("SourceTable", TableDescriptor.forConnector("datag .column("f0", DataTypes.STRING()) .build()) .option(DataGenOptions.ROWS_PER_SECOND, 100) - .build()) + .build()); // Create a sink table (using SQL DDL) tableEnv.executeSql("CREATE TEMPORARY TABLE SinkTable WITH ('connector' = 'blackhole') LIKE SourceTable"); @@ -65,7 +65,7 @@ Table table2 = tableEnv.from("SourceTable"); Table table3 = tableEnv.sqlQuery("SELECT * FROM SourceTable"); // Emit a Table API result Table to a TableSink, same for SQL result -TableResult tableResult = table2.executeInsert("SinkTable"); +TableResult tableResult = table2.insertInto("SinkTable").execute(); ``` {{< /tab >}} {{< tab "Scala" >}} @@ -95,7 +95,7 @@ val table1 = tableEnv.from("SourceTable") val table2 = tableEnv.sqlQuery("SELECT * FROM SourceTable") // Emit a Table API result Table to a TableSink, same for SQL result -val tableResult = table1.executeInsert("SinkTable") +val tableResult = table1.insertInto("SinkTable").execute() ``` {{< /tab >}} {{< tab "Python" >}} @@ -335,7 +335,7 @@ tableEnv.createTable("SourceTableA", sourceDescriptor); tableEnv.createTemporaryTable("SourceTableB", sourceDescriptor); // Using SQL DDL -tableEnv.executeSql("CREATE [TEMPORARY] TABLE MyTable (...) WITH (...)") +tableEnv.executeSql("CREATE [TEMPORARY] TABLE MyTable (...) WITH (...)"); ``` @@ -645,7 +645,9 @@ Table API 和 SQL 查询的混用非常简单因为它们都返回 `Table` 对 请参考文档 [Table Sources & Sinks]({{< ref "docs/dev/table/sourcesSinks" >}}) 以获取更多关于可用 Sink 的信息以及如何自定义 `DynamicTableSink`。 -方法 `Table.executeInsert(String tableName)` 将 `Table` 发送至已注册的 `TableSink`。该方法通过名称在 catalog 中查找 `TableSink` 并确认`Table` schema 和 `TableSink` schema 一致。 +方法 `Table.insertInto(String tableName)` 定义了一个完整的端到端管道将源表中的数据传输到一个被注册的输出表中。 +该方法通过名称在 catalog 中查找输出表并确认 `Table` schema 和输出表 schema 一致。 +可以通过方法 `TablePipeline.explain()` 和 `TablePipeline.execute()` 分别来解释和执行一个数据流管道。 下面的示例演示如何输出 `Table`: @@ -671,10 +673,16 @@ tableEnv.createTemporaryTable("CsvSinkTable", TableDescriptor.forConnector("file .build()); // compute a result Table using Table API operators and/or SQL queries -Table result = ... +Table result = ...; + +// Prepare the insert into pipeline +TablePipeline pipeline = result.insertInto("CsvSinkTable"); + +// Print explain details +pipeline.printExplain(); // emit the result Table to the registered TableSink -result.executeInsert("CsvSinkTable"); +pipeline.execute(); ``` {{< /tab >}} @@ -701,8 +709,14 @@ tableEnv.createTemporaryTable("CsvSinkTable", TableDescriptor.forConnector("file // compute a result Table using Table API operators and/or SQL queries val result: Table = ... +// Prepare the insert into pipeline +val pipeline = result.insertInto("CsvSinkTable") + +// Print explain details +pipeline.printExplain() + // emit the result Table to the registered TableSink -result.executeInsert("CsvSinkTable") +pipeline.execute() ``` {{< /tab >}} @@ -752,9 +766,9 @@ result.execute_insert("CsvSinkTable") Table API 或者 SQL 查询在下列情况下会被翻译: * 当 `TableEnvironment.executeSql()` 被调用时。该方法是用来执行一个 SQL 语句,一旦该方法被调用, SQL 语句立即被翻译。 -* 当 `Table.executeInsert()` 被调用时。该方法是用来将一个表的内容插入到目标表中,一旦该方法被调用, TABLE API 程序立即被翻译。 +* 当 `TablePipeline.execute()` 被调用时。该方法是用来执行一个源表到输出表的数据流,一旦该方法被调用, TABLE API 程序立即被翻译。 * 当 `Table.execute()` 被调用时。该方法是用来将一个表的内容收集到本地,一旦该方法被调用, TABLE API 程序立即被翻译。 -* 当 `StatementSet.execute()` 被调用时。`Table` (通过 `StatementSet.addInsert()` 输出给某个 `Sink`)和 INSERT 语句 (通过调用 `StatementSet.addInsertSql()`)会先被缓存到 `StatementSet` 中,`StatementSet.execute()` 方法被调用时,所有的 sink 会被优化成一张有向无环图。 +* 当 `StatementSet.execute()` 被调用时。`TablePipeline` (通过 `StatementSet.add()` 输出给某个 `Sink`)和 INSERT 语句 (通过调用 `StatementSet.addInsertSql()`)会先被缓存到 `StatementSet` 中,`StatementSet.execute()` 方法被调用时,所有的 sink 会被优化成一张有向无环图。 * 当 `Table` 被转换成 `DataStream` 时(参阅[与 DataStream 集成](#integration-with-datastream))。转换完成后,它就成为一个普通的 DataStream 程序,并会在调用 `StreamExecutionEnvironment.execute()` 时被执行。 {{< top >}} @@ -910,10 +924,10 @@ tEnv.createTemporaryTable("MySink2", TableDescriptor.forConnector("filesystem") StatementSet stmtSet = tEnv.createStatementSet(); Table table1 = tEnv.from("MySource1").where($("word").like("F%")); -stmtSet.addInsert("MySink1", table1); +stmtSet.add(table1.insertInto("MySink1")); Table table2 = table1.unionAll(tEnv.from("MySource2")); -stmtSet.addInsert("MySink2", table2); +stmtSet.add(table2.insertInto("MySink2")); String explanation = stmtSet.explain(); System.out.println(explanation); @@ -954,10 +968,10 @@ tEnv.createTemporaryTable("MySink2", TableDescriptor.forConnector("filesystem") val stmtSet = tEnv.createStatementSet() val table1 = tEnv.from("MySource1").where($"word".like("F%")) -stmtSet.addInsert("MySink1", table1) +stmtSet.add(table1.insertInto("MySink1")) val table2 = table1.unionAll(tEnv.from("MySource2")) -stmtSet.addInsert("MySink2", table2) +stmtSet.add(table2.insertInto("MySink2")) val explanation = stmtSet.explain() println(explanation) diff --git a/docs/content.zh/docs/dev/table/concepts/overview.md b/docs/content.zh/docs/dev/table/concepts/overview.md index 53ef0dc3a7e3d..ed6561c8638a8 100644 --- a/docs/content.zh/docs/dev/table/concepts/overview.md +++ b/docs/content.zh/docs/dev/table/concepts/overview.md @@ -34,106 +34,90 @@ Flink 的 [Table API]({{< ref "docs/dev/table/tableApi" >}}) 和 [SQL]({{< ref " 下面这些页面包含了概念、实际的限制,以及流式数据处理中的一些特定的配置。 -State Management + + +状态管理 ---------------- +流模式下运行的表程序利用了 Flink 作为有状态流处理器的所有能力。 -Table programs that run in streaming mode leverage all capabilities of Flink as a stateful stream -processor. +事实上,一个表程序(Table program)可以配置一个 [state backend]({{< ref "docs/ops/state/state_backends" >}}) +和多个不同的 [checkpoint 选项]({{< ref "docs/dev/datastream/fault-tolerance/checkpointing" >}}) +以处理对不同状态大小和容错需求。这可以对正在运行的 Table API & SQL 管道(pipeline)生成 savepoint,并在这之后用其恢复应用程序的状态。 -In particular, a table program can be configured with a [state backend]({{< ref "docs/ops/state/state_backends" >}}) -and various [checkpointing options]({{< ref "docs/dev/datastream/fault-tolerance/checkpointing" >}}) -for handling different requirements regarding state size and fault tolerance. It is possible to take -a savepoint of a running Table API & SQL pipeline and to restore the application's state at a later -point in time. + -### State Usage +### 状态使用 -Due to the declarative nature of Table API & SQL programs, it is not always obvious where and how much -state is used within a pipeline. The planner decides whether state is necessary to compute a correct -result. A pipeline is optimized to claim as little state as possible given the current set of optimizer -rules. +由于 Table API & SQL 程序是声明式的,管道内的状态会在哪以及如何被使用并不明确。 Planner 会确认是否需要状态来得到正确的计算结果, +管道会被现有优化规则集优化成尽可能少地使用状态。 {{< hint info >}} -Conceptually, source tables are never kept entirely in state. An implementer deals with logical tables -(i.e. [dynamic tables]({{< ref "docs/dev/table/concepts/dynamic_tables" >}})). Their state requirements -depend on the used operations. +从概念上讲, 源表从来不会在状态中被完全保存。 实现者处理的是逻辑表(即[动态表]({{< ref "docs/dev/table/concepts/dynamic_tables" >}}))。 +它们的状态取决于用到的操作。 {{< /hint >}} -Queries such as `SELECT ... FROM ... WHERE` which only consist of field projections or filters are usually -stateless pipelines. However, operations such as joins, aggregations, or deduplications require keeping -intermediate results in a fault-tolerant storage for which Flink's state abstractions are used. +形如 `SELECT ... FROM ... WHERE` 这种只包含字段映射或过滤器的查询的查询语句通常是无状态的管道。 然而诸如 join、 +聚合或去重操作需要在 Flink 抽象的容错存储内保存中间结果。 {{< hint info >}} -Please refer to the individual operator documentation for more details about how much state is required -and how to limit a potentially ever-growing state size. +请参考独立的算子文档来获取更多关于状态需求量和限制潜在增长状态大小的信息。 {{< /hint >}} -For example, a regular SQL join of two tables requires the operator to keep both input tables in state -entirely. For correct SQL semantics, the runtime needs to assume that a matching could occur at any -point in time from both sides. Flink provides [optimized window and interval joins]({{< ref "docs/dev/table/sql/queries/joins" >}}) -that aim to keep the state size small by exploiting the concept of [watermarks]({{< ref "docs/dev/table/concepts/time_attributes" >}}). +例如对两个表进行 join 操作的普通 SQL 需要算子保存两个表的全部输入。基于正确的 SQL 语义,运行时假设两表会在任意时间点进行匹配。 +Flink 提供了 [优化窗口和时段 Join 聚合]({{< ref "docs/dev/table/sql/queries/joins" >}}) +以利用 [watermarks]({{< ref "docs/dev/table/concepts/time_attributes" >}}) 概念来让保持较小的状态规模。 -Another example is the following query that computes the number of clicks per session. +另一个计算每个会话的点击次数的查询语句的例子如下 ```sql SELECT sessionId, COUNT(*) FROM clicks GROUP BY sessionId; ``` -The `sessionId` attribute is used as a grouping key and the continuous query maintains a count -for each `sessionId` it observes. The `sessionId` attribute is evolving over time and `sessionId` -values are only active until the session ends, i.e., for a limited period of time. However, the -continuous query cannot know about this property of `sessionId` and expects that every `sessionId` -value can occur at any point of time. It maintains a count for each observed `sessionId` value. -Consequently, the total state size of the query is continuously growing as more and more `sessionId` -values are observed. +`sessionId` 是用于分组的键,连续查询(Continuous Query)维护了每个观察到的 `sessionId` 次数。 `sessionId` 属性随着时间逐步演变, +且 `sessionId` 的值只活跃到会话结束(即在有限的时间周期内)。然而连续查询无法得知sessionId的这个性质, +并且预期每个 `sessionId` 值会在任何时间点上出现。这维护了每个可见的 `sessionId` 值。因此总状态量会随着 `sessionId` 的发现不断地增长。 + + + +#### 空闲状态维持时间 -#### Idle State Retention Time +*空间状态位置时间*参数 [`table.exec.state.ttl`]({{< ref "docs/dev/table/config" >}}#table-exec-state-ttl) +定义了状态的键在被更新后要保持多长时间才被移除。在之前的查询例子中,`sessionId` 的数目会在配置的时间内未更新时立刻被移除。 -The *Idle State Retention Time* parameter [`table.exec.state.ttl`]({{< ref "docs/dev/table/config" >}}#table-exec-state-ttl) -defines for how long the state of a key is retained without being updated before it is removed. -For the previous example query, the count of a`sessionId` would be removed as soon as it has not -been updated for the configured period of time. +通过移除状态的键,连续查询会完全忘记它曾经见过这个键。如果一个状态带有曾被移除状态的键被处理了,这条记录将被认为是 +对应键的第一条记录。上述例子中意味着 `sessionId` 会再次从 `0` 开始计数。 -By removing the state of a key, the continuous query completely forgets that it has seen this key -before. If a record with a key, whose state has been removed before, is processed, the record will -be treated as if it was the first record with the respective key. For the example above this means -that the count of a `sessionId` would start again at `0`. + -### Stateful Upgrades and Evolution +### 状态化更新与演化 -Table programs that are executed in streaming mode are intended as *standing queries* which means they -are defined once and are continuously evaluated as static end-to-end pipelines. +表程序在流模式下执行将被视为*标准查询*,这意味着它们被定义一次后将被一直视为静态的端到端 (end-to-end) 管道 -In case of stateful pipelines, any change to both the query or Flink's planner might lead to a completely -different execution plan. This makes stateful upgrades and the evolution of table programs challenging -at the moment. The community is working on improving those shortcomings. +对于这种状态化的管道,对查询和Flink的Planner的改动都有可能导致完全不同的执行计划。这让表程序的状态化的升级和演化在目前而言 +仍具有挑战,社区正致力于改进这一缺点。 -For example, by adding a filter predicate, the optimizer might decide to reorder joins or change the -schema of an intermediate operator. This prevents restoring from a savepoint due to either changed -topology or different column layout within the state of an operator. +例如为了添加过滤谓词,优化器可能决定重排 join 或改变内部算子的 schema。 这会阻碍从 savepoint 的恢复,因为其被改变的拓扑和 +算子状态的列布局差异。 -The query implementer must ensure that the optimized plans before and after the change are compatible. -Use the `EXPLAIN` command in SQL or `table.explain()` in Table API to [get insights]({{< ref "docs/dev/table/common" >}}#explaining-a-table). +查询实现者需要确保改变在优化计划前后是兼容的,在 SQL 中使用 `EXPLAIN` 或在 Table API 中使用 `table.explain()` +可[获取详情]({{< ref "docs/dev/table/common" >}}#explaining-a-table)。 -Since new optimizer rules are continuously added, and operators become more efficient and specialized, -also the upgrade to a newer Flink version could lead to incompatible plans. +由于新的优化器规则正不断地被添加,算子变得更加高效和专用,升级到更新的Flink版本可能造成不兼容的计划。 {{< hint warning >}} -Currently, the framework cannot guarantee that state can be mapped from a savepoint to a new table -operator topology. +当前框架无法保证状态可以从 savepoint 映射到新的算子拓扑上。 -In other words: Savepoints are only supported if both the query and the Flink version remain constant. +换言之: Savepoint 只在查询语句和版本保持恒定的情况下被支持。 {{< /hint >}} -Since the community rejects contributions that modify the optimized plan and the operator topology -in a patch version (e.g. from `1.13.1` to `1.13.2`), it should be safe to upgrade a Table API & SQL -pipeline to a newer bug fix release. However, major-minor upgrades from (e.g. from `1.12` to `1.13`) -are not supported. +由于社区拒绝在版本补丁(如 `1.13.1` 至 `1.13.2`)上对优化计划和算子拓扑进行修改的贡献,对一个 Table API & SQL 管道 +升级到新的 bug fix 发行版应当是安全的。然而主次(major-minor)版本的更新(如 `1.12` 至 `1.13`)不被支持。 + +由于这两个缺点(即修改查询语句和修改Flink版本),我们推荐实现调查升级后的表程序是否可以在切换到实时数据前,被历史数据"暖机" +(即被初始化)。Flink社区正致力于 [混合源]({{< ref "docs/connectors/datastream/hybridsource" >}}) 来让切换变得尽可能方便。 + -For both shortcomings (i.e. modified query and modified Flink version), we recommend to investigate -whether the state of an updated table program can be "warmed up" (i.e. initialized) with historical -data again before switching to real-time data. The Flink community is working on a [hybrid source]({{< ref "docs/connectors/datastream/hybridsource" >}}) -to make this switching as convenient as possible. + 接下来? ----------------- diff --git a/docs/content.zh/docs/dev/table/config.md b/docs/content.zh/docs/dev/table/config.md index 70939a9f8d74f..07f69c7a76a54 100644 --- a/docs/content.zh/docs/dev/table/config.md +++ b/docs/content.zh/docs/dev/table/config.md @@ -46,7 +46,7 @@ Table 和 SQL API 的默认配置能够确保结果准确,同时也提供可 {{< tab "Java" >}} ```java // instantiate table environment -TableEnvironment tEnv = ... +TableEnvironment tEnv = ...; // access flink configuration TableConfig configuration = tEnv.getConfig(); diff --git a/docs/content.zh/docs/dev/table/data_stream_api.md b/docs/content.zh/docs/dev/table/data_stream_api.md index ff12448050eec..9e3078505fc86 100644 --- a/docs/content.zh/docs/dev/table/data_stream_api.md +++ b/docs/content.zh/docs/dev/table/data_stream_api.md @@ -467,6 +467,8 @@ import org.apache.flink.table.api.bridge.scala._ {{< /tab >}} {{< /tabs >}} +请查阅[配置]({{< ref "docs/dev/configuration/overview" >}})小节了解更多细节。 + ### Configuration The `TableEnvironment` will adopt all configuration options from the passed `StreamExecutionEnvironment`. @@ -596,25 +598,25 @@ pipeline or a statement set: ```java // execute with explicit sink -tableEnv.from("InputTable").executeInsert("OutputTable") +tableEnv.from("InputTable").insertInto("OutputTable").execute(); -tableEnv.executeSql("INSERT INTO OutputTable SELECT * FROM InputTable") +tableEnv.executeSql("INSERT INTO OutputTable SELECT * FROM InputTable"); tableEnv.createStatementSet() - .addInsert("OutputTable", tableEnv.from("InputTable")) - .addInsert("OutputTable2", tableEnv.from("InputTable")) - .execute() + .add(tableEnv.from("InputTable").insertInto("OutputTable")) + .add(tableEnv.from("InputTable").insertInto("OutputTable2")) + .execute(); tableEnv.createStatementSet() .addInsertSql("INSERT INTO OutputTable SELECT * FROM InputTable") .addInsertSql("INSERT INTO OutputTable2 SELECT * FROM InputTable") - .execute() + .execute(); // execute with implicit local sink -tableEnv.from("InputTable").execute().print() +tableEnv.from("InputTable").execute().print(); -tableEnv.executeSql("SELECT * FROM InputTable").print() +tableEnv.executeSql("SELECT * FROM InputTable").print(); ``` To combine both execution behaviors, every call to `StreamTableEnvironment.toDataStream` @@ -627,17 +629,17 @@ these "external parts". // (1) // adds a branch with a printing sink to the StreamExecutionEnvironment -tableEnv.toDataStream(table).print() +tableEnv.toDataStream(table).print(); // (2) // executes a Table API end-to-end pipeline as a Flink job and prints locally, // thus (1) has still not been executed -table.execute().print() +table.execute().print(); // executes the DataStream API pipeline with the sink defined in (1) as a // Flink job, (2) was already running before -env.execute() +env.execute(); ``` {{< top >}} @@ -2560,12 +2562,12 @@ TableDescriptor sinkDescriptor = TableDescriptor.forConnector("print").build(); // add a pure Table API pipeline Table tableFromSource = tableEnv.from(sourceDescriptor); -statementSet.addInsert(sinkDescriptor, tableFromSource); +statementSet.add(tableFromSource.insertInto(sinkDescriptor)); // use table sinks for the DataStream API pipeline DataStream dataStream = env.fromElements(1, 2, 3); Table tableFromStream = tableEnv.fromDataStream(dataStream); -statementSet.addInsert(sinkDescriptor, tableFromStream); +statementSet.add(tableFromStream.insertInto(sinkDescriptor)); // attach both pipelines to StreamExecutionEnvironment // (the statement set will be cleared after calling this method) @@ -2611,12 +2613,12 @@ val sinkDescriptor = TableDescriptor.forConnector("print").build // add a pure Table API pipeline val tableFromSource = tableEnv.from(sourceDescriptor) -statementSet.addInsert(sinkDescriptor, tableFromSource) +statementSet.add(tableFromSource.insertInto(sinkDescriptor)) // use table sinks for the DataStream API pipeline val dataStream = env.fromElements(1, 2, 3) val tableFromStream = tableEnv.fromDataStream(dataStream) -statementSet.addInsert(sinkDescriptor, tableFromStream) +statementSet.add(tableFromStream.insertInto(sinkDescriptor)) // attach both pipelines to StreamExecutionEnvironment // (the statement set will be cleared calling this method) @@ -2772,7 +2774,7 @@ In particular, these parts might not be well integrated into many recent new fea {{< tab "Java" >}} ```java StreamTableEnvironment tableEnv = ...; -DataStream> stream = ... +DataStream> stream = ...; Table table2 = tableEnv.fromDataStream(stream, $("myLong"), $("myString")); ``` @@ -2908,7 +2910,7 @@ Flink 的 DataStream API 支持多样的数据类型。 ```java StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section; -DataStream> stream = ... +DataStream> stream = ...; // convert DataStream into Table with field "myLong" only Table table = tableEnv.fromDataStream(stream, $("myLong")); @@ -2960,7 +2962,7 @@ table = t_env.from_data_stream(stream, col('my_long'), col('my_int')) ```java StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section -DataStream> stream = ... +DataStream> stream = ...; // convert DataStream into Table with field "f1" only Table table = tableEnv.fromDataStream(stream, $("f1")); @@ -3023,7 +3025,7 @@ Flink 将基础数据类型(`Integer`、`Double`、`String`)或者通用数 ```java StreamTableEnvironment tableEnv = ...; -DataStream stream = ... +DataStream stream = ...; // Convert DataStream into Table with field name "myLong" Table table = tableEnv.fromDataStream(stream, $("myLong")); @@ -3081,7 +3083,7 @@ tuple 的 DataStream 都能被转换成表。 ```java StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section -DataStream> stream = ... +DataStream> stream = ...; // convert DataStream into Table with renamed field names "myLong", "myString" (position-based) Table table = tableEnv.fromDataStream(stream, $("myLong"), $("myString")); @@ -3176,7 +3178,7 @@ Flink 支持 POJO 类型作为复合类型。确定 POJO 类型的规则记录 StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section // Person is a POJO with fields "name" and "age" -DataStream stream = ... +DataStream stream = ...; // convert DataStream into Table with renamed fields "myAge", "myName" (name-based) Table table = tableEnv.fromDataStream(stream, $("age").as("myAge"), $("name").as("myName")); @@ -3225,7 +3227,7 @@ Row 类型的字段映射支持基于名称和基于位置两种方式。 StreamTableEnvironment tableEnv = ...; // DataStream of Row with two fields "name" and "age" specified in `RowTypeInfo` -DataStream stream = ... +DataStream stream = ...; // Convert DataStream into Table with renamed field names "myName", "myAge" (position-based) Table table = tableEnv.fromDataStream(stream, $("myName"), $("myAge")); diff --git a/docs/content.zh/docs/dev/table/functions/systemFunctions.md b/docs/content.zh/docs/dev/table/functions/systemFunctions.md index 3a660189fd253..69ab489a9625e 100644 --- a/docs/content.zh/docs/dev/table/functions/systemFunctions.md +++ b/docs/content.zh/docs/dev/table/functions/systemFunctions.md @@ -206,7 +206,7 @@ Known Limitations: ```java table .groupBy("withColumns(1 to 3)") - .select("withColumns(a to b), myUDAgg(myUDF(withColumns(5 to 20)))") + .select("withColumns(a to b), myUDAgg(myUDF(withColumns(5 to 20)))"); ``` {{< /tab >}} {{< tab "Scala" >}} diff --git a/docs/content.zh/docs/dev/table/overview.md b/docs/content.zh/docs/dev/table/overview.md index b3038cb0814dd..3365f0d0dafa6 100644 --- a/docs/content.zh/docs/dev/table/overview.md +++ b/docs/content.zh/docs/dev/table/overview.md @@ -33,74 +33,11 @@ Table API 和 SQL 两种 API 是紧密集成的,以及 DataStream API。你可 ## Table 程序依赖 -取决于你使用的编程语言,选择 Java 或者 Scala API 来构建你的 Table API 和 SQL 程序: +您需要将 Table API 作为依赖项添加到项目中,以便用 Table API 和 SQL 定义数据管道。 -{{< tabs "94f8aceb-507f-4c8f-977e-df00fe903203" >}} -{{< tab "Java" >}} -```xml - - org.apache.flink - flink-table-api-java-bridge{{< scala_version >}} - {{< version >}} - provided - -``` -{{< /tab >}} -{{< tab "Scala" >}} -```xml - - org.apache.flink - flink-table-api-scala-bridge{{< scala_version >}} - {{< version >}} - provided - -``` -{{< /tab >}} -{{< tab "Python" >}} -{{< stable >}} -```bash -$ python -m pip install apache-flink {{< version >}} -``` -{{< /stable >}} -{{< unstable >}} -```bash -$ python -m pip install apache-flink -``` -{{< /unstable >}} -{{< /tab >}} -{{< /tabs >}} +有关如何为 Java 和 Scala 配置这些依赖项的更多细节,请查阅[项目配置]({{< ref "docs/dev/configuration/overview" >}})小节。 -除此之外,如果你想在 IDE 本地运行你的程序,你需要添加下面的模块,具体用哪个取决于你使用哪个 Planner: - -```xml - - org.apache.flink - flink-table-planner{{< scala_version >}} - {{< version >}} - provided - - - org.apache.flink - flink-streaming-scala{{< scala_version >}} - {{< version >}} - provided - -``` - -### 扩展依赖 - -如果你想实现[自定义格式或连接器]({{< ref "docs/dev/table/sourcesSinks" >}}) 用于(反)序列化行或一组[用户定义的函数]({{< ref "docs/dev/table/functions/udfs" >}}),下面的依赖就足够了,编译出来的 jar 文件可以直接给 SQL Client 使用: - -```xml - - org.apache.flink - flink-table-common - {{< version >}} - provided - -``` - -{{< top >}} +如果您使用 Python,请查阅 [Python API]({{< ref "docs/dev/python/overview" >}}) 文档。 接下来? ----------------- diff --git a/docs/content.zh/docs/dev/table/sourcesSinks.md b/docs/content.zh/docs/dev/table/sourcesSinks.md index 4f1c5080d68ce..d1940bab57929 100644 --- a/docs/content.zh/docs/dev/table/sourcesSinks.md +++ b/docs/content.zh/docs/dev/table/sourcesSinks.md @@ -106,6 +106,33 @@ that the planner can handle. {{< top >}} + +Project Configuration +--------------------- + +If you want to implement a custom connector or a custom format, the following dependency is usually +sufficient: + +{{< artifact_tabs flink-table-common withProvidedScope >}} + +If you want to develop a connector that needs to bridge with DataStream APIs (i.e. if you want to adapt +a DataStream connector to the Table API), you need to add this dependency: + +{{< artifact_tabs flink-table-api-java-bridge withProvidedScope >}} + +When developing the connector/format, we suggest shipping both a thin JAR and an uber JAR, so users +can easily load the uber JAR in the SQL client or in the Flink distribution and start using it. +The uber JAR should include all the third-party dependencies of the connector, +excluding the table dependencies listed above. + +{{< hint warning >}} +You should not depend on `flink-table-planner{{< scala_version >}}` in production code. +With the new module `flink-table-planner-loader` introduced in Flink 1.15, the +application's classpath will not have direct access to `org.apache.flink.table.planner` classes anymore. +If you need a feature available only internally within the `org.apache.flink.table.planner` package and subpackages, please open an issue. +To learn more, check out [Anatomy of Table Dependencies]({{< ref "docs/dev/configuration/advanced" >}}#anatomy-of-table-dependencies). +{{< /hint >}} + Extension Points ---------------- diff --git a/docs/content.zh/docs/dev/table/sql/queries/match_recognize.md b/docs/content.zh/docs/dev/table/sql/queries/match_recognize.md index e6b11fc99af4c..9f3e15b0edd0d 100644 --- a/docs/content.zh/docs/dev/table/sql/queries/match_recognize.md +++ b/docs/content.zh/docs/dev/table/sql/queries/match_recognize.md @@ -84,7 +84,7 @@ Flink 的 `MATCH_RECOGNIZE` 子句实现是一个完整标准子集。仅支持 ``` -或者,也可以将依赖项添加到集群的 classpath(查看 [dependency section]({{< ref "docs/dev/datastream/project-configuration" >}}) 获取更多相关依赖信息)。 +或者,也可以将依赖项添加到集群的 classpath(查看 [dependency section]({{< ref "docs/dev/configuration/overview" >}}) 获取更多相关依赖信息)。 如果你想在 [SQL Client]({{< ref "docs/dev/table/sqlClient" >}}) 中使用 `MATCH_RECOGNIZE` 子句,你无需执行任何操作,因为默认情况下包含所有依赖项。 diff --git a/docs/content.zh/docs/dev/table/sqlClient.md b/docs/content.zh/docs/dev/table/sqlClient.md index 2239121c50c8f..6f6067010b36a 100644 --- a/docs/content.zh/docs/dev/table/sqlClient.md +++ b/docs/content.zh/docs/dev/table/sqlClient.md @@ -368,16 +368,17 @@ When execute queries or insert statements, please enter the interactive mode or ### Dependencies -The SQL Client does not require to setup a Java project using Maven or SBT. Instead, you can pass the -dependencies as regular JAR files that get submitted to the cluster. You can either specify each JAR -file separately (using `--jar`) or define entire library directories (using `--library`). For +The SQL Client does not require setting up a Java project using Maven, Gradle, or sbt. Instead, you +can pass the dependencies as regular JAR files that get submitted to the cluster. You can either specify +each JAR file separately (using `--jar`) or define entire library directories (using `--library`). For connectors to external systems (such as Apache Kafka) and corresponding data formats (such as JSON), Flink provides **ready-to-use JAR bundles**. These JAR files can be downloaded for each release from the Maven central repository. -The full list of offered SQL JARs and documentation about how to use them can be found on the [connection to external systems page]({{< ref "docs/connectors/table/overview" >}}). +The full list of offered SQL JARs can be found on the [connection to external systems page]({{< ref "docs/connectors/table/overview" >}}). -{{< top >}} +You can refer to the [configuration]({{< ref "docs/dev/configuration/connector" >}}) section for +information on how to configure connector and format dependencies. Use SQL Client to submit job ---------------------------- diff --git a/docs/content.zh/docs/dev/table/tableApi.md b/docs/content.zh/docs/dev/table/tableApi.md index 860464a0f6b84..65a9f9d87a257 100644 --- a/docs/content.zh/docs/dev/table/tableApi.md +++ b/docs/content.zh/docs/dev/table/tableApi.md @@ -1455,7 +1455,9 @@ result3 = table.order_by(table.a.asc).offset(10).fetch(5) {{< label Batch >}} {{< label Streaming >}} -和 SQL 查询中的 `INSERT INTO` 子句类似,该方法执行对已注册的输出表的插入操作。`executeInsert()` 方法将立即提交执行插入操作的 Flink job。 +和 SQL 查询中的 `INSERT INTO` 子句类似,该方法执行对已注册的输出表的插入操作。 +`insertInto()` 方法会将 `INSERT INTO` 转换为一个 `TablePipeline`。 +该数据流可以用 `TablePipeline.explain()` 来解释,用 `TablePipeline.execute()` 来执行。 输出表必须已注册在 TableEnvironment(详见表连接器)中。此外,已注册表的 schema 必须与查询中的 schema 相匹配。 @@ -1463,13 +1465,13 @@ result3 = table.order_by(table.a.asc).offset(10).fetch(5) {{< tab "Java" >}} ```java Table orders = tableEnv.from("Orders"); -orders.executeInsert("OutOrders"); +orders.insertInto("OutOrders").execute(); ``` {{< /tab >}} {{< tab "Scala" >}} ```scala val orders = tableEnv.from("Orders") -orders.executeInsert("OutOrders") +orders.insertInto("OutOrders").execute() ``` {{< /tab >}} {{< tab "Python" >}} diff --git a/docs/content.zh/docs/dev/table/tuning.md b/docs/content.zh/docs/dev/table/tuning.md index 5d3111a5a77ea..9aa82529a1cce 100644 --- a/docs/content.zh/docs/dev/table/tuning.md +++ b/docs/content.zh/docs/dev/table/tuning.md @@ -59,7 +59,7 @@ Window TVF aggregation buffer records in [managed memory]({{< ref "docs/deployme {{< tab "Java" >}} ```java // instantiate table environment -TableEnvironment tEnv = ... +TableEnvironment tEnv = ...; // access flink configuration TableConfig configuration = tEnv.getConfig(); @@ -88,11 +88,11 @@ configuration.set("table.exec.mini-batch.size", "5000") // the maximum number of t_env = ... # access flink configuration -configuration = t_env.get_config().get_configuration(); +configuration = t_env.get_config() # set low-level key-value options -configuration.set_string("table.exec.mini-batch.enabled", "true"); # enable mini-batch optimization -configuration.set_string("table.exec.mini-batch.allow-latency", "5 s"); # use 5 seconds to buffer input records -configuration.set_string("table.exec.mini-batch.size", "5000"); # the maximum number of records can be buffered by each aggregate operator task +configuration.set("table.exec.mini-batch.enabled", "true") # enable mini-batch optimization +configuration.set("table.exec.mini-batch.allow-latency", "5 s") # use 5 seconds to buffer input records +configuration.set("table.exec.mini-batch.size", "5000") # the maximum number of records can be buffered by each aggregate operator task ``` {{< /tab >}} {{< /tabs >}} @@ -121,7 +121,7 @@ GROUP BY color {{< tab "Java" >}} ```java // instantiate table environment -TableEnvironment tEnv = ... +TableEnvironment tEnv = ...; // access flink configuration Configuration configuration = tEnv.getConfig().getConfiguration(); @@ -152,12 +152,12 @@ configuration.set("table.optimizer.agg-phase-strategy", "TWO_PHASE") // enable t t_env = ... # access flink configuration -configuration = t_env.get_config().get_configuration(); +configuration = t_env.get_config() # set low-level key-value options -configuration.set_string("table.exec.mini-batch.enabled", "true"); # local-global aggregation depends on mini-batch is enabled -configuration.set_string("table.exec.mini-batch.allow-latency", "5 s"); -configuration.set_string("table.exec.mini-batch.size", "5000"); -configuration.set_string("table.optimizer.agg-phase-strategy", "TWO_PHASE"); # enable two-phase, i.e. local-global aggregation +configuration.set("table.exec.mini-batch.enabled", "true") # local-global aggregation depends on mini-batch is enabled +configuration.set("table.exec.mini-batch.allow-latency", "5 s") +configuration.set("table.exec.mini-batch.size", "5000") +configuration.set("table.optimizer.agg-phase-strategy", "TWO_PHASE") # enable two-phase, i.e. local-global aggregation ``` {{< /tab >}} {{< /tabs >}} @@ -206,7 +206,7 @@ GROUP BY day {{< tab "Java" >}} ```java // instantiate table environment -TableEnvironment tEnv = ... +TableEnvironment tEnv = ...; tEnv.getConfig() .set("table.optimizer.distinct-agg.split.enabled", "true"); // enable distinct agg split @@ -226,8 +226,7 @@ tEnv.getConfig # instantiate table environment t_env = ... -t_env.get_config() - .set("table.optimizer.distinct-agg.split.enabled", "true"); # enable distinct agg split +t_env.get_config().set("table.optimizer.distinct-agg.split.enabled", "true") # enable distinct agg split ``` {{< /tab >}} {{< /tabs >}} diff --git a/docs/content.zh/docs/flinkDev/ide_setup.md b/docs/content.zh/docs/flinkDev/ide_setup.md index f17b32d36519a..a4c985917b328 100644 --- a/docs/content.zh/docs/flinkDev/ide_setup.md +++ b/docs/content.zh/docs/flinkDev/ide_setup.md @@ -28,7 +28,7 @@ under the License. # 导入 Flink 到 IDE 中 -以下章节描述了如何将 Flink 项目导入到 IDE 中以进行 Flink 本身的源码开发。有关 Flink 程序编写的信息,请参阅 [Java API]({{< ref "docs/dev/datastream/project-configuration" >}}) 和 [Scala API]({{< ref "docs/dev/datastream/project-configuration" >}}) 快速入门指南。 +以下章节描述了如何将 Flink 项目导入到 IDE 中以进行 Flink 本身的源码开发。有关 Flink 程序编写的信息,请参阅 [Java API]({{< ref "docs/dev/configuration/overview" >}}) 和 [Scala API]({{< ref "docs/dev/configuration/overview" >}}) 快速入门指南。 {{< hint info >}} 每当你的 IDE 无法正常工作时,请优先尝试使用 Maven 命令行(`mvn clean package -DskipTests`),因为它可能是由于你的 IDE 中存在错误或未正确设置。 diff --git a/docs/content.zh/docs/learn-flink/datastream_api.md b/docs/content.zh/docs/learn-flink/datastream_api.md index fdad73b9588fe..a032c96d65b7f 100644 --- a/docs/content.zh/docs/learn-flink/datastream_api.md +++ b/docs/content.zh/docs/learn-flink/datastream_api.md @@ -1,6 +1,6 @@ --- title: DataStream API 简介 -weight: 2 +weight: 3 type: docs --- + +# Checkpoints vs. Savepoints + +## Overview + +Conceptually, Flink's [savepoints]({{< ref "docs/ops/state/savepoints" >}}) are different from [checkpoints]({{< ref "docs/ops/state/checkpoints" >}}) +in a way that's analogous to how backups are different from recovery logs in traditional database systems. + +The primary purpose of checkpoints is to provide a recovery mechanism in case of unexpected job failures. +A [checkpoint's lifecycle]({{< ref "docs/dev/datastream/fault-tolerance/checkpointing" >}}) is managed by Flink, +i.e. a checkpoint is created, owned, and released by Flink - without user interaction. +Because checkpoints are being triggered often, and are relied upon for failure recovery, the two main design goals for the checkpoint implementation are +i) being as lightweight to create and ii) being as fast to restore from as possible. +Optimizations towards those goals can exploit certain properties, e.g., that the job code doesn't change between the execution attempts. + +{{< hint info >}} +- Checkpoints are automatically deleted if the application is terminated by the user +(except if checkpoints are explicitly configured to be retained). +- Checkpoints are stored in state backend-specific (native) data format (may be incremental depending on the specific backend). +{{< /hint >}} + +Although [savepoints]({{< ref "docs/ops/state/savepoints" >}}) are created internally with the same mechanisms as +checkpoints, they are conceptually different and can be a bit more expensive to produce and restore from. Their design focuses +more on portability and operational flexibility, especially with respect to changes to the job. +The use case for savepoints is for planned, manual operations. For example, this could be an update of your Flink version, changing your job graph, and so on. + +{{< hint info >}} +- Savepoints are created, owned and deleted solely by the user. +That means, Flink does not delete savepoints neither after job termination nor after +restore. +- Savepoints are stored in a state backend independent (canonical) format (Note: Since Flink 1.15, savepoints can be also stored in +the backend-specific [native]({{< ref "docs/ops/state/savepoints" >}}#savepoint-format) format which is faster to create +and restore but comes with some limitations. +{{< /hint >}} + +### Capabilities and limitations +The following table gives an overview of capabilities and limitations for the various types of savepoints and +checkpoints. +- ✓ - Flink fully support this type of the snapshot +- x - Flink doesn't support this type of the snapshot +- ! - While these operations currently work, Flink doesn't officially guarantee support for them, so there is a certain level of risk associated with them + +| Operation | Canonical Savepoint | Native Savepoint | Aligned Checkpoint | Unaligned Checkpoint | +|:----------------------------------|:--------------------|:-----------------|:-------------------|:---------------------| +| State backend change | ✓ | x | x | x | +| State Processor API (writing) | ✓ | x | x | x | +| State Processor API (reading) | ✓ | ! | ! | x | +| Self-contained and relocatable | ✓ | ✓ | x | x | +| Schema evolution | ✓ | ! | ! | ! | +| Arbitrary job upgrade | ✓ | ✓ | ✓ | x | +| Non-arbitrary job upgrade | ✓ | ✓ | ✓ | x | +| Flink minor version upgrade | ✓ | ✓ | ✓ | x | +| Flink bug/patch version upgrade | ✓ | ✓ | ✓ | ✓ | +| Rescaling | ✓ | ✓ | ✓ | ✓ | + +- [State backend change]({{< ref "docs/ops/state/state_backends" >}}) - configuring a different State Backend than was used when taking the snapshot. +- [State Processor API (writing)]({{< ref "docs/libs/state_processor_api" >}}#writing-new-savepoints) - the ability to create a new snapshot of this type via the State Processor API. +- [State Processor API (reading)]({{< ref "docs/libs/state_processor_api" >}}#reading-state) - the ability to read states from an existing snapshot of this type via the State Processor API. +- Self-contained and relocatable - the one snapshot folder contains everything it needs for recovery +and it doesn't depend on other snapshots which means it can be easily moved to another place if needed. +- [Schema evolution]({{< ref "docs/dev/datastream/fault-tolerance/serialization/schema_evolution" >}}) - the *state* data type can be changed if it uses a serializer that supports schema evolution (e.g., POJOs and Avro types) +- Arbitrary job upgrade - the snapshot can be restored even if the [partitioning types]({{< ref "docs/dev/datastream/operators/overview" >}}#physical-partitioning)(rescale, rebalance, map, etc.) +or in-flight record types for the existing operators have changed. +- Non-arbitrary job upgrade - restoring the snapshot is possible with updated operators if the job graph topology and in-flight record types remain unchanged. +- Flink minor version upgrade - restoring a snapshot taken with an older minor version of Flink (1.x → 1.y). +- Flink bug/patch version upgrade - restoring a snapshot taken with an older patch version of Flink (1.14.x → 1.14.y). +- Rescaling - restoring the snapshot with a different parallelism than was used during the snapshot creation. + + +{{< top >}} diff --git a/docs/content.zh/docs/ops/state/savepoints.md b/docs/content.zh/docs/ops/state/savepoints.md index 51785b67e563d..11f336898d93c 100644 --- a/docs/content.zh/docs/ops/state/savepoints.md +++ b/docs/content.zh/docs/ops/state/savepoints.md @@ -27,7 +27,7 @@ under the License. # Savepoints -## 什么是 Savepoint ? Savepoint 与 Checkpoint 有什么不同? +## 什么是 Savepoint ? Savepoint 是依据 Flink [checkpointing 机制]({{< ref "docs/learn-flink/fault_tolerance" >}})所创建的流作业执行状态的一致镜像。 你可以使用 Savepoint 进行 Flink 作业的停止与重启、fork 或者更新。 Savepoint 由两部分组成:稳定存储(列入 HDFS,S3,...) 上包含二进制文件的目录(通常很大),和元数据文件(相对较小)。 稳定存储上的文件表示作业执行状态的数据镜像。 Savepoint 的元数据文件以(相对路径)的形式包含(主要)指向作为 Savepoint 一部分的稳定存储上的所有文件的指针。 @@ -35,11 +35,7 @@ Savepoint 是依据 Flink [checkpointing 机制]({{< ref "docs/learn-flink/fault **注意:** 为了允许程序和 Flink 版本之间的升级,请务必查看以下有关分配算子 ID 的部分 。 {{< /hint >}} -从概念上讲, Flink 的 Savepoint 与 Checkpoint 的不同之处类似于传统数据库中的备份与恢复日志之间的差异。 Checkpoint 的主要目的是为意外失败的作业提供恢复机制。 Checkpoint 的生命周期由 Flink 管理,即 Flink 创建,管理和删除 Checkpoint - 无需用户交互。 作为一种恢复和定期触发的方法,Checkpoint 实现有两个设计目标:i)轻量级创建和 ii)尽可能快地恢复。 可能会利用某些特定的属性来达到这个,例如, 工作代码在执行尝试之间不会改变。 在用户终止作业后,通常会删除 Checkpoint(除非明确配置为保留的 Checkpoint)。 - - 与此相反、Savepoint 由用户创建,拥有和删除。 他们的用例是计划的,手动备份和恢复。 例如,升级 Flink 版本,调整用户逻辑,改变并行度,以及进行红蓝部署等。 当然,Savepoint 必须在作业停止后继续存在。 从概念上讲,Savepoint 的生成,恢复成本可能更高一些,Savepoint 更多地关注可移植性和对前面提到的作业更改的支持。 - -除去这些概念上的差异,Checkpoint 和 Savepoint 的当前实现基本上使用相同的代码并生成相同的格式。然而,目前有一个例外,我们可能会在未来引入更多的差异。例外情况是使用 RocksDB 状态后端的增量 Checkpoint。他们使用了一些 RocksDB 内部格式,而不是 Flink 的本机 Savepoint 格式。这使他们成为了与 Savepoint 相比,更轻量级的 Checkpoint 机制的第一个实例。 +To make proper use of savepoints, it's important to understand the differences between [checkpoints]({{< ref "docs/ops/state/checkpoints" >}}) and savepoints which is described in [checkpoints vs. savepoints]({{< ref "docs/ops/state/checkpoints_vs_savepoints" >}}). ## 分配算子 ID @@ -157,10 +153,55 @@ $ bin/flink run -s :savepointPath [:runArgs] 默认情况下,resume 操作将尝试将 Savepoint 的所有状态映射回你要还原的程序。 如果删除了运算符,则可以通过 `--allowNonRestoredState`(short:`-n`)选项跳过无法映射到新程序的状态: +#### Restore 模式 + +`Restore 模式` 决定了在 restore 之后谁拥有Savepoint 或者 [externalized checkpoint]({{< ref "docs/ops/state/checkpoints" >}}/#resuming-from-a-retained-checkpoint)的文件的所有权。在这种语境下 Savepoint 和 externalized checkpoint 的行为相似。 +这里我们将它们都称为“快照”,除非另有明确说明。 + +如前所述,restore 模式决定了谁来接管我们从中恢复的快照文件的所有权。快照可被用户或者 Flink 自身拥有。如果快照归用户所有,Flink 不会删除其中的文件,而且 Flink 不能依赖该快照中文件的存在,因为它可能在 Flink 的控制之外被删除。 + +每种 restore 模式都有特定的用途。尽管如此,我们仍然认为默认的 *NO_CLAIM* 模式在大多数情况下是一个很好的折中方案,因为它在提供明确的所有权归属的同时只给恢复后第一个 checkpoint 带来较小的代价。 + +你可以通过如下方式指定 restore 模式: ```shell -$ bin/flink run -s :savepointPath -n [:runArgs] +$ bin/flink run -s :savepointPath -restoreMode :mode -n [:runArgs] ``` +**NO_CLAIM (默认的)** + +在 *NO_CLAIM* 模式下,Flink 不会接管快照的所有权。它会将快照的文件置于用户的控制之中,并且永远不会删除其中的任何文件。该模式下可以从同一个快照上启动多个作业。 + +为保证 Flink 不会依赖于该快照的任何文件,它会强制第一个(成功的) checkpoint 为全量 checkpoint 而不是增量的。这仅对`state.backend: rocksdb` 有影响,因为其他 backend 总是创建全量 checkpoint。 + +一旦第一个全量的 checkpoint 完成后,所有后续的 checkpoint 会照常创建。所以,一旦一个 checkpoint 成功制作,就可以删除原快照。在此之前不能删除原快照,因为没有任何完成的 checkpoint,Flink 会在故障时尝试从初始的快照恢复。 + +
+ {{< img src="/fig/restore-mode-no_claim.svg" alt="NO_CLAIM restore mode" width="70%" >}} +
+ +**CLAIM** + +另一个可选的模式是 *CLAIM* 模式。该模式下 Flink 将声称拥有快照的所有权,并且本质上将其作为 checkpoint 对待:控制其生命周期并且可能会在其永远不会被用于恢复的时候删除它。因此,手动删除快照和从同一个快照上启动两个作业都是不安全的。Flink 会保持[配置数量]({{< ref "docs/dev/datastream/fault-tolerance/checkpointing" >}}/#state-checkpoints-num-retained)的 checkpoint。 + +
+ {{< img src="/fig/restore-mode-claim.svg" alt="CLAIM restore mode" width="70%" >}} +
+ +{{< hint info >}} +**注意:** +1. Retained checkpoints 被存储在 `//chk-` 这样的目录中。Flink 不会接管 `/` 目录的所有权,而只会接管 `chk-` 的所有权。Flink 不会删除旧作业的目录。 + +2. [Native](#savepoint-format) 格式支持增量的 RocksDB savepoints。对于这些 savepoints,Flink 将所有 SST 存储在 savepoints 目录中。这意味着这些 savepoints 是自包含和目录可移动的。然而,在 CLAIM 模式下恢复时,后续的 checkpoints 可能会复用一些 SST 文件,这反过来会阻止在 savepoints 被清理时删除 savepoints 目录。 Flink 之后运行期间可能会删除复用的SST 文件,但不会删除 savepoints 目录。因此,如果在 CLAIM 模式下恢复,Flink 可能会留下一个空的 savepoints 目录。 +{{< /hint >}} + +**LEGACY** + +Legacy 模式是 Flink 在 1.15 之前的工作方式。该模式下 Flink 永远不会删除初始恢复的 checkpoint。同时,用户也不清楚是否可以删除它。导致该的问题原因是, Flink 会在用来恢复的 checkpoint 之上创建增量的 checkpoint,因此后续的 checkpoint 都有可能会依赖于用于恢复的那个 checkpoint。总而言之,恢复的 checkpoint 的所有权没有明确的界定。 + +
+ {{< img src="/fig/restore-mode-legacy.svg" alt="LEGACY restore mode" width="70%" >}} +
+ ### 删除 Savepoint ```shell diff --git a/docs/content.zh/docs/ops/state/state_backends.md b/docs/content.zh/docs/ops/state/state_backends.md index e6d6764a0f3d2..6c8c4593686b0 100644 --- a/docs/content.zh/docs/ops/state/state_backends.md +++ b/docs/content.zh/docs/ops/state/state_backends.md @@ -38,7 +38,9 @@ under the License. 在启动 CheckPoint 机制时,状态会随着 CheckPoint 而持久化,以防止数据丢失、保障恢复时的一致性。 状态内部的存储格式、状态在 CheckPoint 时如何持久化以及持久化在哪里均取决于选择的 **State Backend**。 -# 可用的 State Backends + + +## 可用的 State Backends Flink 内置了以下这些开箱即用的 state backends : @@ -48,6 +50,8 @@ Flink 内置了以下这些开箱即用的 state backends : 如果不设置,默认使用 HashMapStateBackend。 + + ### HashMapStateBackend 在 *HashMapStateBackend* 内部,数据以 Java 对象的形式存储在堆中。 Key/value 形式的状态和窗口算子会持有一个 hash table,其中存储着状态值、触发器。 @@ -59,6 +63,10 @@ HashMapStateBackend 的适用场景: 建议同时将 [managed memory]({{< ref "docs/deployment/memory/mem_setup_tm" >}}#managed-memory) 设为0,以保证将最大限度的内存分配给 JVM 上的用户代码。 +与 EmbeddedRocksDBStateBackend 不同的是,由于 HashMapStateBackend 将数据以对象形式存储在堆中,因此重用这些对象数据是不安全的。 + + + ### EmbeddedRocksDBStateBackend EmbeddedRocksDBStateBackend 将正在运行中的状态数据保存在 [RocksDB](http://rocksdb.org) 数据库中,RocksDB 数据库默认将数据存储在 TaskManager 的数据目录。 @@ -79,6 +87,7 @@ EmbeddedRocksDBStateBackend 的适用场景: 注意,你可以保留的状态大小仅受磁盘空间的限制。与状态存储在内存中的 HashMapStateBackend 相比,EmbeddedRocksDBStateBackend 允许存储非常大的状态。 然而,这也意味着使用 EmbeddedRocksDBStateBackend 将会使应用程序的最大吞吐量降低。 所有的读写都必须序列化、反序列化操作,这个比基于堆内存的 state backend 的效率要低很多。 +同时因为存在这些序列化、反序列化操作,重用放入 EmbeddedRocksDBStateBackend 的对象是安全的。 请同时参考 [Task Executor 内存配置]({{< ref "docs/deployment/memory/mem_tuning" >}}#rocksdb-state-backend) 中关于 EmbeddedRocksDBStateBackend 的建议。 @@ -88,7 +97,9 @@ EmbeddedRocksDBStateBackend 是目前唯一支持增量 CheckPoint 的 State Bac 每个 slot 中的 RocksDB instance 的内存大小是有限制的,详情请见 [这里]({{< ref "docs/ops/state/large_state_tuning" >}})。 -# 选择合适的 State Backend + + +## 选择合适的 State Backend 在选择 `HashMapStateBackend` 和 `RocksDB` 的时候,其实就是在性能与可扩展性之间权衡。`HashMapStateBackend` 是非常快的,因为每个状态的读取和算子对于 objects 的更新都是在 Java 的 heap 上;但是状态的大小受限于集群中可用的内存。 另一方面,`RocksDB` 可以根据可用的 disk 空间扩展,并且只有它支持增量 snapshot。 @@ -99,11 +110,15 @@ EmbeddedRocksDBStateBackend 是目前唯一支持增量 CheckPoint 的 State Bac 从 1.13 版本开始,所有的 state backends 都会生成一种普适的格式。因此,如果想切换 state backend 的话,那么最好先升级你的 Flink 版本,在新版本中生成 savepoint,在这之后你才可以使用一个不同的 state backend 来读取并恢复它。 {{< /hint >}} + + ## 设置 State Backend 如果没有明确指定,将使用 jobmanager 做为默认的 state backend。你能在 **flink-conf.yaml** 中为所有 Job 设置其他默认的 State Backend。 每一个 Job 的 state backend 配置会覆盖默认的 state backend 配置,如下所示: + + ### 设置每个 Job 的 State Backend `StreamExecutionEnvironment` 可以对每个 Job 的 State Backend 进行设置,如下所示: @@ -138,6 +153,7 @@ env.setStateBackend(new HashMapStateBackend()) **注意:** 由于 RocksDB 是 Flink 默认分发包的一部分,所以如果你没在代码中使用 RocksDB,则不需要添加此依赖。而且可以在 `flink-conf.yaml` 文件中通过 `state.backend` 配置 State Backend,以及更多的 [checkpointing]({{< ref "docs/deployment/config" >}}#checkpointing) 和 [RocksDB 特定的]({{< ref "docs/deployment/config" >}}#rocksdb-state-backend) 参数。 {{< /hint >}} + ### 设置默认的(全局的) State Backend @@ -163,10 +179,14 @@ state.backend: filesystem state.checkpoints.dir: hdfs://namenode:40010/flink/checkpoints ``` -# RocksDB State Backend 进阶 + + +## RocksDB State Backend 进阶 *该小节描述 RocksDB state backend 的更多细节* + + ### 增量快照 RocksDB 支持*增量快照*。不同于产生一个包含所有数据的全量备份,增量快照中只包含自上一次快照完成之后被修改的记录,因此可以显著减少快照完成的耗时。 @@ -181,6 +201,8 @@ RocksDB 支持*增量快照*。不同于产生一个包含所有数据的全量 需要注意的是,一旦启用了增量快照,网页上展示的 `Checkpointed Data Size` 只代表增量上传的数据量,而不是一次快照的完整数据量。 + + ### 内存管理 Flink 致力于控制整个进程的内存消耗,以确保 Flink 任务管理器(TaskManager)有良好的内存使用,从而既不会在容器(Docker/Kubernetes, Yarn等)环境中由于内存超用被杀掉,也不会因为内存利用率过低导致不必要的数据落盘或是缓存命中率下降,致使性能下降。 @@ -210,6 +232,8 @@ Flink还提供了两个参数来控制*写路径*(MemTable)和*读路径*( 或者可以复用上述 cache/write-buffer-manager 机制,但将内存大小设置为与 Flink 的托管内存大小无关的固定大小(通过 `state.backend.rocksdb.memory.fixed-per-slot` 选项)。 注意在这两种情况下,用户都需要确保在 JVM 之外有足够的内存可供 RocksDB 使用。 + + ### 计时器(内存 vs. RocksDB) 计时器(Timer)用于安排稍后的操作(基于事件时间或处理时间),例如触发窗口或回调 `ProcessFunction`。 @@ -220,6 +244,8 @@ Flink还提供了两个参数来控制*写路径*(MemTable)和*读路径*( 注意 *在 RocksDB state backend 中使用基于堆的计时器的组合当前不支持计时器状态的异步快照。其他状态(如 keyed state)可以被异步快照。* + + ### 开启 RocksDB 原生监控指标 您可以选择使用 Flink 的监控指标系统来汇报 RocksDB 的原生指标,并且可以选择性的指定特定指标进行汇报。 @@ -229,7 +255,7 @@ Flink还提供了两个参数来控制*写路径*(MemTable)和*读路径*( **注意:** 启用 RocksDB 的原生指标可能会对应用程序的性能产生负面影响。 {{< /hint >}} -### 列族(ColumnFamily)级别的预定义选项 +#### 列族(ColumnFamily)级别的预定义选项 注意 在引入 [RocksDB 使用托管内存](#memory-management) 功能后,此机制应限于在*专家调优*或*故障处理*中使用。 @@ -246,7 +272,7 @@ Flink还提供了两个参数来控制*写路径*(MemTable)和*读路径*( RocksDB State Backend 会将 [这里定义]({{< ref "docs/deployment/config" >}}#advanced-rocksdb-state-backends-options) 的所有配置项全部加载。 因此您可以简单的通过关闭 RocksDB 使用托管内存的功能并将需要的设置选项加入配置文件来配置底层的列族选项。 -### 通过 RocksDBOptionsFactory 配置 RocksDB 选项 +#### 通过 RocksDBOptionsFactory 配置 RocksDB 选项 注意 在引入 [RocksDB 使用托管内存](#memory-management) 功能后,此机制应限于在*专家调优*或*故障处理*中使用。 @@ -303,77 +329,75 @@ public class MyOptionsFactory implements ConfigurableRocksDBOptionsFactory { {{< top >}} -## Enabling Changelog + + +## 开启 Changelog + +{{< hint warning >}} 该功能处于实验状态。 {{< /hint >}} + +{{< hint warning >}} 开启 Changelog 可能会给您的应用带来性能损失。(见下文) {{< /hint >}} + + + +### 介绍 -{{< hint warning >}} This feature is in experimental status. {{< /hint >}} +Changelog 是一项旨在减少 checkpointing 时间的功能,因此也可以减少 exactly-once 模式下的端到端延迟。 -{{< hint warning >}} Enabling Changelog may have a negative performance impact on your application (see below). {{< /hint >}} +一般情况下 checkpoint 的持续时间受如下因素影响: -### Introduction +1. Barrier 到达和对齐时间,可以通过 [Unaligned checkpoints]({{< ref "docs/ops/state/checkpointing_under_backpressure#unaligned-checkpoints" >}}) 和 [Buffer debloating]({{< ref "docs/ops/state/checkpointing_under_backpressure#buffer-debloating" >}}) 解决。 -Changelog is a feature that aims to decrease checkpointing time and, therefore, end-to-end latency in exactly-once mode. +2. 快照制作时间(所谓同步阶段), 可以通过异步快照解决(如[上文]({{< + ref "#the-embeddedrocksdbstatebackend">}})所述)。 -Most commonly, checkpoint duration is affected by: +3. 快照上传时间(异步阶段)。 -1. Barrier travel time and alignment, addressed by - [Unaligned checkpoints]({{< ref "docs/ops/state/checkpointing_under_backpressure#unaligned-checkpoints" >}}) - and [Buffer debloating]({{< ref "docs/ops/state/checkpointing_under_backpressure#buffer-debloating" >}}) -2. Snapshot creation time (so-called synchronous phase), addressed by asynchronous snapshots (mentioned [above]({{< - ref "#the-embeddedrocksdbstatebackend">}})) -4. Snapshot upload time (asynchronous phase) +可以用[增量 checkpoints]({{< ref "#incremental-checkpoints" >}}) 来减少上传时间。但是,大多数支持增量checkpoint的状态后端会定期执行合并类型的操作,这会导致除了新的变更之外还要重新上传旧状态。在大规模部署中,每次 checkpoint 中至少有一个 task 上传大量数据的可能性往往非常高。 -Upload time can be decreased by [incremental checkpoints]({{< ref "#incremental-checkpoints" >}}). -However, most incremental state backends perform some form of compaction periodically, which results in re-uploading the -old state in addition to the new changes. In large deployments, the probability of at least one task uploading lots of -data tends to be very high in every checkpoint. +开启 Changelog 功能之后,Flink 会不断上传状态变更并形成 changelog。创建 checkpoint 时,只有 changelog 中的相关部分需要上传。而配置的状态后端则会定期在后台进行快照,快照成功上传后,相关的changelog 将会被截断。 -With Changelog enabled, Flink uploads state changes continuously and forms a changelog. On checkpoint, only the relevant -part of this changelog needs to be uploaded. The configured state backend is snapshotted in the -background periodically. Upon successful upload, the changelog is truncated. +基于此,异步阶段的持续时间减少(另外因为不需要将数据刷新到磁盘,同步阶段持续时间也减少了),特别是长尾延迟得到了改善。 -As a result, asynchronous phase duration is reduced, as well as synchronous phase - because no data needs to be flushed -to disk. In particular, long-tail latency is improved. +但是,资源使用会变得更高: -However, resource usage is higher: +- 将会在 DFS 上创建更多文件 +- 将可能在 DFS 上残留更多文件(这将在 FLINK-25511 和 FLINK-25512 之后的新版本中被解决) +- 将使用更多的 IO 带宽用来上传状态变更 +- 将使用更多 CPU 资源来序列化状态变更 +- Task Managers 将会使用更多内存来缓存状态变更 -- more files are created on DFS -- more files can be left undeleted DFS (this will be addressed in the future versions in FLINK-25511 and FLINK-25512) -- more IO bandwidth is used to upload state changes -- more CPU used to serialize state changes -- more memory used by Task Managers to buffer state changes +另一项需要考虑的事情是恢复时间。取决于 `state.backend.changelog.periodic-materialize.interval` 的设置,changelog 可能会变得冗长,因此重放会花费更多时间。即使这样,恢复时间加上 checkpoint 持续时间仍然可能低于不开启 changelog 功能的时间,从而在故障恢复的情况下也能提供更低的端到端延迟。当然,取决于上述时间的实际比例,有效恢复时间也有可能会增加。 -Recovery time is another thing to consider. Depending on the `state.backend.changelog.periodic-materialize.interval` -setting, the changelog can become lengthy and replaying it may take more time. However, recovery time combined with -checkpoint duration will likely still be lower than in non-changelog setups, providing lower end-to-end latency even in -failover case. However, it's also possible that the effective recovery time will increase, depending on the actual ratio -of the aforementioned times. +有关更多详细信息,请参阅 [FLIP-158](https://cwiki.apache.org/confluence/display/FLINK/FLIP-158%3A+Generalized+incremental+checkpoints)。 -For more details, see [FLIP-158](https://cwiki.apache.org/confluence/display/FLINK/FLIP-158%3A+Generalized+incremental+checkpoints). + -### Installation +### 安装 -Changelog JARs are included into the standard Flink distribution. +标准的 Flink 发行版包含 Changelog 所需要的 JAR包。 -Make sure to [add]({{< ref "docs/deployment/filesystems/overview" >}}) the necessary filesystem plugins. +请确保[添加]({{< ref "docs/deployment/filesystems/overview" >}})所需的文件系统插件。 -### Configuration + -Here is an example configuration in YAML: +### 配置 + +这是 YAML 中的示例配置: ```yaml state.backend.changelog.enabled: true -state.backend.changelog.storage: filesystem # currently, only filesystem and memory (for tests) are supported -dstl.dfs.base-path: s3:// # similar to state.checkpoints.dir +state.backend.changelog.storage: filesystem # 当前只支持 filesystem 和 memory(仅供测试用) +dstl.dfs.base-path: s3:// # 类似于 state.checkpoints.dir ``` -Please keep the following defaults (see [limitations](#limitations)): +请将如下配置保持默认值 (参见[限制](#limitations)): ```yaml execution.checkpointing.max-concurrent-checkpoints: 1 state.backend.local-recovery: false ``` -Please refer to the [configuration section]({{< ref "docs/deployment/config#state-changelog-options" >}}) for other options. +有关其他配置选项,请参阅[配置]({{< ref "docs/deployment/config#state-changelog-options" >}})部分。 -Changelog can also be enabled or disabled per job programmatically: +也可以通过编程方式为每个作业开启或关闭 Changelog: {{< tabs >}} {{< tab "Java" >}} ```java @@ -395,39 +419,45 @@ env.enable_changelog_statebackend(true) {{< /tab >}} {{< /tabs >}} -### Monitoring + + +### 监控 + +[此处]({{< ref "docs/ops/metrics#state-changelog" >}})列出了可用的指标。 -Available metrics are listed [here]({{< ref "docs/ops/metrics#changelog" >}}). +如果 task 因写状态变更而被反压,他将在 UI 中被显示为忙碌(红色)。 -If a task is backpressured by writing state changes, it will be shown as busy (red) in the UI. + -### Upgrading existing jobs +### 升级现有作业 -**Enabling Changelog** +**开启 Changelog** -Resuming only from savepoints in canonical format is supported: -- given an existing non-changelog job -- take a [savepoint]({{< ref "docs/ops/state/savepoints#resuming-from-savepoints" >}}) (canonical format is the default) -- alter configuration (enable Changelog) -- resume from the taken snapshot +仅支持从标准格式的 savepoint 恢复: +- 给定一个没有开启 Changelog 的作业 +- 创建一个 [savepoint]({{< ref "docs/ops/state/savepoints#resuming-from-savepoints" >}}) (默认为标准格式) +- 更改配置(开启 Changelog) +- 从创建的 snapshot 恢复 -**Disabling Changelog** +**关闭 Changelog** -Resuming only from [savepoints]({{< ref "docs/ops/state/savepoints#resuming-from-savepoints" >}}) -is supported. Resuming from [checkpoints]({{< ref "docs/ops/state/checkpoints#resuming-from-a-retained-checkpoint" >}}) -is planned in the future versions. +仅支持从 [savepoints]({{< ref "docs/ops/state/savepoints#resuming-from-savepoints" >}}) 恢复。从 [checkpoints]({{< ref "docs/ops/state/checkpoints#resuming-from-a-retained-checkpoint" >}}) 恢复计划在未来版本中支持。 -**State migration** (including changing TTL) is currently not supported +当前不支持**状态迁移**(包括改变 TTL)。 -### Limitations - - At most one concurrent checkpoint - - Local recovery not supported - - As of Flink 1.15, only `filesystem` changelog implementation is available - - State migration (including changing TTL) is currently not supported -- [NO_CLAIM]({{< ref "docs/deployment/config#execution-savepoint-restore-mode" >}}) mode not supported + + +### 限制 +- 最多同时创建一个 checkpoint +- 本地恢复暂不支持 +- 到 Flink 1.15 为止, 只有 `filesystem` changelog 实现可用 +- 尚不支持状态迁移(包括修改 TTL) +- 尚不支持 [NO_CLAIM]({{< ref "docs/deployment/config#execution-savepoint-restore-mode" >}}) 模式 {{< top >}} + + ## 自旧版本迁移 从 **Flink 1.13** 版本开始,社区改进了 state backend 的公开类,进而帮助用户更好理解本地状态存储和 checkpoint 存储的区分。 diff --git a/docs/content.zh/docs/ops/upgrading.md b/docs/content.zh/docs/ops/upgrading.md index 73d4c3cf7604b..b12111e43bd9b 100644 --- a/docs/content.zh/docs/ops/upgrading.md +++ b/docs/content.zh/docs/ops/upgrading.md @@ -216,6 +216,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: 1.11.x 1.12.x 1.13.x + 1.14.x + 1.15.x Limitations @@ -235,6 +237,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: + + The maximum parallelism of a job that was migrated from Flink 1.1.x to 1.2.x+ is currently fixed as the parallelism of the job. This means that the parallelism can not be increased after migration. This limitation might be removed in a future bugfix release. @@ -254,6 +258,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O When migrating from Flink 1.2.x to Flink 1.3.x+, changing parallelism at the same time is not supported. Users have to first take a savepoint after migrating to Flink 1.3.x+, and then change @@ -279,6 +285,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O Migrating from Flink 1.3.0 to Flink 1.4.[0,1] will fail if the savepoint contains Scala case classes. Users have to directly migrate to 1.4.2+ instead. @@ -296,6 +304,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -313,6 +323,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O There is a known issue with resuming broadcast state created with 1.5.x in versions 1.6.x up to 1.6.2, and 1.7.0: FLINK-11087. Users upgrading to 1.6.x or 1.7.x series need to directly migrate to minor versions higher than 1.6.2 and 1.7.0, @@ -333,6 +345,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -350,6 +364,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -367,6 +383,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -384,6 +402,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -401,6 +421,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -418,6 +440,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -435,6 +459,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O + O + O @@ -452,8 +478,48 @@ Savepoints are compatible across Flink versions as indicated by the table below: O + O + O Don't upgrade from 1.12.x to 1.13.x with an unaligned checkpoint. Please use a savepoint for migrating. + + 1.14.x + + + + + + + + + + + + + + O + O + + + + 1.15.x + + + + + + + + + + + + + + + O + + diff --git a/docs/content/docs/concepts/stateful-stream-processing.md b/docs/content/docs/concepts/stateful-stream-processing.md index f8c2a77dcb5f6..7f139b021bb70 100644 --- a/docs/content/docs/concepts/stateful-stream-processing.md +++ b/docs/content/docs/concepts/stateful-stream-processing.md @@ -312,6 +312,9 @@ mechanism for this. Savepoints are similar to checkpoints except that they are **triggered by the user** and **don't automatically expire** when newer checkpoints are completed. +To make proper use of savepoints, it's important to understand the differences between +[checkpoints]({{< ref "docs/ops/state/checkpoints" >}}) and [savepoints]({{< ref "docs/ops/state/savepoints" >}}) +which is described in [checkpoints vs. savepoints]({{< ref "docs/ops/state/checkpoints_vs_savepoints" >}}). {{< top >}} diff --git a/docs/content/docs/concepts/time.md b/docs/content/docs/concepts/time.md index 093583e709390..ef68ccbf47365 100644 --- a/docs/content/docs/concepts/time.md +++ b/docs/content/docs/concepts/time.md @@ -84,8 +84,8 @@ one can refer to different notions of *time*: out-of-order or late events, or when reprocessing historic data. For example, an hourly event time window will contain all records that carry an event timestamp that falls into that hour, regardless of the order in which they - arrive, or when they are processed. (See the section on [late - events](#late-elements) for more information.) + arrive, or when they are processed. (See the section on [lateness](#lateness) + for more information.) Note that sometimes when event time programs are processing live data in real-time, they will use some *processing time* operations in order to diff --git a/docs/content/docs/connectors/dataset/formats/avro.md b/docs/content/docs/connectors/dataset/formats/avro.md index 732058765e563..5adb6691ca905 100644 --- a/docs/content/docs/connectors/dataset/formats/avro.md +++ b/docs/content/docs/connectors/dataset/formats/avro.md @@ -51,7 +51,7 @@ DataSet usersDS = env.createInput(users); Note that `User` is a POJO generated by Avro. Flink also allows to perform string-based key selection of these POJOs. For example: ```java -usersDS.keyBy("name") +usersDS.keyBy("name"); ``` diff --git a/docs/content/docs/connectors/dataset/formats/hadoop.md b/docs/content/docs/connectors/dataset/formats/hadoop.md index b517c986f54de..be5205472c52b 100644 --- a/docs/content/docs/connectors/dataset/formats/hadoop.md +++ b/docs/content/docs/connectors/dataset/formats/hadoop.md @@ -117,7 +117,7 @@ The following example shows how to use Hadoop's `TextOutputFormat`. ```java // Obtain the result we want to emit -DataSet> hadoopResult = [...] +DataSet> hadoopResult = [...]; // Set up the Hadoop TextOutputFormat. HadoopOutputFormat hadoopOF = diff --git a/docs/content/docs/connectors/datastream/elasticsearch.md b/docs/content/docs/connectors/datastream/elasticsearch.md index c8342aed895c6..d467bb391b22e 100644 --- a/docs/content/docs/connectors/datastream/elasticsearch.md +++ b/docs/content/docs/connectors/datastream/elasticsearch.md @@ -134,7 +134,6 @@ private static IndexRequest createIndexRequest(String element) { return Requests.indexRequest() .index("my-index") - .type("my-type") .id(element) .source(json); } @@ -195,7 +194,7 @@ def createIndexRequest(element: (String)): IndexRequest = { "data" -> element.asInstanceOf[AnyRef] ) - Requests.indexRequest.index("my-index").`type`("my-type").source(mapAsJavaMap(json)) + Requests.indexRequest.index("my-index").source(mapAsJavaMap(json)) } ``` {{< /tab >}} @@ -214,10 +213,7 @@ flushes of the buffered actions in progress. ### Elasticsearch Sinks and Fault Tolerance -By default, the Flink Elasticsearch Sink will not provide any strong delivery guarantees. -Users have the option to enable at-least-once semantics for the Elasticsearch sink. - -With Flink’s checkpointing enabled, the Flink Elasticsearch Sink can guarantee +With Flink’s checkpointing enabled, the Flink Elasticsearch Sink guarantees at-least-once delivery of action requests to Elasticsearch clusters. It does so by waiting for all pending action requests in the `BulkProcessor` at the time of checkpoints. This effectively assures that all requests before the @@ -226,34 +222,13 @@ proceeding to process more records sent to the sink. More details on checkpoints and fault tolerance are in the [fault tolerance docs]({{< ref "docs/learn-flink/fault_tolerance" >}}). -To use fault tolerant Elasticsearch Sinks, at-least-once delivery has to be configured and checkpointing of the topology needs to be enabled at the execution environment: +To use fault tolerant Elasticsearch Sinks, checkpointing of the topology needs to be enabled at the execution environment: {{< tabs "d00d1e93-4844-40d7-b0ec-9ec37e73145e" >}} {{< tab "Java" >}} -Elasticsearch 6: -```java -final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); -env.enableCheckpointing(5000); // checkpoint every 5000 msecs - -Elasticsearch6SinkBuilder sinkBuilder = new Elasticsearch6SinkBuilder() - .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE) - .setHosts(new HttpHost("127.0.0.1", 9200, "http")) - .setEmitter( - (element, context, indexer) -> - indexer.add(createIndexRequest(element))); -``` - -Elasticsearch 7: ```java final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.enableCheckpointing(5000); // checkpoint every 5000 msecs - -Elasticsearch7SinkBuilder sinkBuilder = new Elasticsearch7SinkBuilder() - .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE) - .setHosts(new HttpHost("127.0.0.1", 9200, "http")) - .setEmitter( - (element, context, indexer) -> - indexer.add(createIndexRequest(element))); ``` {{< /tab >}} {{< tab "Scala" >}} @@ -261,28 +236,21 @@ Elasticsearch 6: ```scala val env = StreamExecutionEnvironment.getExecutionEnvironment() env.enableCheckpointing(5000) // checkpoint every 5000 msecs - -val sinkBuilder = new Elasticsearch6SinkBuilder[String] - .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE) - .setHosts(new HttpHost("127.0.0.1", 9200, "http")) - .setEmitter((element: String, context: SinkWriter.Context, indexer: RequestIndexer) => - indexer.add(createIndexRequest(element))) -``` - -Elasticsearch 7: -```scala -val env = StreamExecutionEnvironment.getExecutionEnvironment() -env.enableCheckpointing(5000) // checkpoint every 5000 msecs - -val sinkBuilder = new Elasticsearch7SinkBuilder[String] - .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE) - .setHosts(new HttpHost("127.0.0.1", 9200, "http")) - .setEmitter((element: String, context: SinkWriter.Context, indexer: RequestIndexer) => - indexer.add(createIndexRequest(element))) ``` {{< /tab >}} {{< /tabs >}} +

+IMPORTANT: Checkpointing is not enabled by default but the default delivery guarantee is AT_LEAST_ONCE. +This causes the sink to buffer requests until it either finishes or the BulkProcessor flushes automatically. +By default, the BulkProcessor will flush after 1000 added Actions. To configure the processor to flush more frequently, please refer to the BulkProcessor configuration section. +

+ +

+Using UpdateRequests with deterministic ids and the upsert method it is possible to achieve exactly-once semantics in Elasticsearch when AT_LEAST_ONCE delivery is configured for the connector. +

+ + ### Handling Failing Elasticsearch Requests Elasticsearch action requests may fail due to a variety of reasons, including diff --git a/docs/content/docs/connectors/datastream/filesystem.md b/docs/content/docs/connectors/datastream/filesystem.md index 4c6a8c258dbde..1a116e0a9c1ef 100644 --- a/docs/content/docs/connectors/datastream/filesystem.md +++ b/docs/content/docs/connectors/datastream/filesystem.md @@ -65,10 +65,10 @@ You can start building a File Source via one of the following API calls: {{< tab "Java" >}} ```java // reads the contents of a file from a file stream. -FileSource.forRecordStreamFormat(StreamFormat,Path...) +FileSource.forRecordStreamFormat(StreamFormat,Path...); // reads batches of records from a file at a time -FileSource.forBulkFileFormat(BulkFormat,Path...) +FileSource.forBulkFileFormat(BulkFormat,Path...); ``` {{< /tab >}} {{< /tabs >}} diff --git a/docs/content/docs/connectors/datastream/formats/avro.md b/docs/content/docs/connectors/datastream/formats/avro.md index 1b2ffef4e812e..f6956e2888b66 100644 --- a/docs/content/docs/connectors/datastream/formats/avro.md +++ b/docs/content/docs/connectors/datastream/formats/avro.md @@ -51,7 +51,7 @@ DataStream usersDS = env.createInput(users); Note that `User` is a POJO generated by Avro. Flink also allows to perform string-based key selection of these POJOs. For example: ```java -usersDS.keyBy("name") +usersDS.keyBy("name"); ``` diff --git a/docs/content/docs/connectors/datastream/formats/parquet.md b/docs/content/docs/connectors/datastream/formats/parquet.md index 71ed416f4da68..8a6f9f9971d0e 100644 --- a/docs/content/docs/connectors/datastream/formats/parquet.md +++ b/docs/content/docs/connectors/datastream/formats/parquet.md @@ -30,7 +30,7 @@ under the License. Flink supports reading [Parquet](https://parquet.apache.org/) files, producing {{< javadoc file="org/apache/flink/table/data/RowData.html" name="Flink RowData">}} and producing [Avro](https://avro.apache.org/) records. -To use the format you need to add the Flink Parquet dependency to your project: +To use the format you need to add the `flink-parquet` dependency to your project: ```xml @@ -39,46 +39,78 @@ To use the format you need to add the Flink Parquet dependency to your project: {{< version >}} ``` - -This format is compatible with the new Source that can be used in both batch and streaming modes. + +To read Avro records, you will need to add the `parquet-avro` dependency: + +```xml + + org.apache.parquet + parquet-avro + 1.12.2 + true + + + org.apache.hadoop + hadoop-client + + + it.unimi.dsi + fastutil + + + +``` + +This format is compatible with the new Source that can be used in both batch and streaming execution modes. Thus, you can use this format for two kinds of data: -- Bounded data -- Unbounded data: monitors a directory for new files that appear -## Flink RowData +- Bounded data: lists all files and reads them all. +- Unbounded data: monitors a directory for new files that appear. -#### Bounded data example +{{< hint info >}} +When you start a File Source it is configured for bounded data by default. +To configure the File Source for unbounded data, you must additionally call +`AbstractFileSource.AbstractFileSourceBuilder.monitorContinuously(Duration)`. +{{< /hint >}} -In this example, you will create a DataStream containing Parquet records as Flink RowDatas. The schema is projected to read only the specified fields ("f7", "f4" and "f99"). -Flink will read records in batches of 500 records. The first boolean parameter specifies that timestamp columns will be interpreted as UTC. -The second boolean instructs the application that the projected Parquet fields names are case-sensitive. -There is no watermark strategy defined as records do not contain event timestamps. +**Vectorized reader** ```java -final LogicalType[] fieldTypes = - new LogicalType[] { - new DoubleType(), new IntType(), new VarCharType() - }; -final ParquetColumnarRowInputFormat format = - new ParquetColumnarRowInputFormat<>( - new Configuration(), - RowType.of(fieldTypes, new String[] {"f7", "f4", "f99"}), - 500, - false, - true); -final FileSource source = - FileSource.forBulkFileFormat(format, /* Flink Path */) - .build(); -final DataStream stream = - env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); +// Parquet rows are decoded in batches +FileSource.forBulkFileFormat(BulkFormat,Path...) + +// Monitor the Paths to read data as unbounded data +FileSource.forBulkFileFormat(BulkFormat,Path...) + .monitorContinuously(Duration.ofMillis(5L)) + .build(); + ``` -#### Unbounded data example +**Avro Parquet reader** -In this example, you will create a DataStream containing Parquet records as Flink RowDatas that will -infinitely grow as new files are added to the directory. It will monitor for new files each second. -The schema is projected to read only the specified fields ("f7", "f4" and "f99"). +```java + +// Parquet rows are decoded in batches +FileSource.forRecordStreamFormat(StreamFormat,Path...) + +// Monitor the Paths to read data as unbounded data +FileSource.forRecordStreamFormat(StreamFormat,Path...) + .monitorContinuously(Duration.ofMillis(5L)) + .build(); + + +``` + +{{< hint info >}} +Following examples are all configured for bounded data. +To configure the File Source for unbounded data, you must additionally call +`AbstractFileSource.AbstractFileSourceBuilder.monitorContinuously(Duration)`. +{{< /hint >}} + +## Flink RowData + +In this example, you will create a DataStream containing Parquet records as Flink RowDatas. The schema is projected to read only the specified fields ("f7", "f4" and "f99"). Flink will read records in batches of 500 records. The first boolean parameter specifies that timestamp columns will be interpreted as UTC. The second boolean instructs the application that the projected Parquet fields names are case-sensitive. There is no watermark strategy defined as records do not contain event timestamps. @@ -98,7 +130,6 @@ final ParquetColumnarRowInputFormat format = true); final FileSource source = FileSource.forBulkFileFormat(format, /* Flink Path */) - .monitorContinuously(Duration.ofSeconds(1L)) .build(); final DataStream stream = env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); @@ -131,9 +162,7 @@ This example uses an Avro schema example similar to the one described in the [of This schema defines a record representing a user with three fields: name, favoriteNumber, and favoriteColor. You can find more details at [record specification](https://avro.apache.org/docs/1.10.0/spec.html#schema_record) for how to define an Avro schema. -#### Bounded data example - -In this example, you will create a DataStream containing Parquet records as Avro Generic records. +In the following example, you will create a DataStream containing Parquet records as Avro Generic records. It will parse the Avro schema based on the JSON string. There are many other ways to parse a schema, e.g. from java.io.File or java.io.InputStream. Please refer to [Avro Schema](https://avro.apache.org/docs/1.10.0/api/java/org/apache/avro/Schema.html) for details. After that, you will create an `AvroParquetRecordFormat` via `AvroParquetReaders` for Avro Generic records. @@ -163,37 +192,6 @@ final DataStream stream = env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); ``` -#### Unbounded data example - -This example is similar to the bounded batch example. The application monitors for new files every second -and reads Avro Generic records from Parquet files infinitely as new files are added to the directory. -```java -// parsing avro schema -final Schema schema = - new Schema.Parser() - .parse( - "{\"type\": \"record\", " - + "\"name\": \"User\", " - + "\"fields\": [\n" - + " {\"name\": \"name\", \"type\": \"string\" },\n" - + " {\"name\": \"favoriteNumber\", \"type\": [\"int\", \"null\"] },\n" - + " {\"name\": \"favoriteColor\", \"type\": [\"string\", \"null\"] }\n" - + " ]\n" - + " }"); - -final FileSource source = - FileSource.forRecordStreamFormat( - AvroParquetReaders.forGenericRecord(schema), /* Flink Path */) - .monitorContinuously(Duration.ofSeconds(1L)) - .build(); - -final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.enableCheckpointing(10L); - -final DataStream stream = - env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); -``` - ### Specific record Based on the previously defined schema, you can generate classes by leveraging Avro code generation. @@ -202,9 +200,7 @@ You can either use `avro-tools.jar` to generate code manually or you could use t code generation on any .avsc files present in the configured source directory. Please refer to [Avro Getting Started](https://avro.apache.org/docs/1.10.0/gettingstartedjava.html) for more information. -#### Bounded data example - -This example uses the example schema [testdata.avsc](https://github.com/apache/flink/blob/master/flink-formats/flink-parquet/src/test/resources/avro/testdata.avsc): +The following example uses the example schema [testdata.avsc](https://github.com/apache/flink/blob/master/flink-formats/flink-parquet/src/test/resources/avro/testdata.avsc): ```json lines [ @@ -247,26 +243,6 @@ final DataStream stream = env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); ``` -#### Unbounded data example - -This example, similar to the bounded batch example, uses the same generated Address Java class -and monitors for the new files every second to read Avro Specific records from Parquet files -infinitely as new files are added to the directory. - -```java -final FileSource source = - FileSource.forRecordStreamFormat( - AvroParquetReaders.forSpecificRecord(Address.class), /* Flink Path */) - .monitorContinuously(Duration.ofSeconds(1L)) - .build(); - -final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.enableCheckpointing(10L); - -final DataStream stream = - env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); -``` - ### Reflect record Beyond Avro Generic and Specific record that requires a predefined Avro schema, @@ -274,8 +250,6 @@ Flink also supports creating a DataStream from Parquet files based on existing J In this case, Avro will use Java reflection to generate schemas and protocols for these POJO classes. Java types are mapped to Avro schemas, please refer to the [Avro reflect](https://avro.apache.org/docs/1.10.0/api/java/index.html) documentation for more details. -#### Bounded data example - This example uses a simple Java POJO class [Datum](https://github.com/apache/flink/blob/master/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/avro/Datum.java): ```java @@ -329,22 +303,94 @@ final DataStream stream = env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); ``` -#### Unbounded data example +#### Prerequisite for Parquet files + +In order to support reading Avro reflect records, the Parquet file must contain specific meta information. +The Avro schema used for creating the Parquet data must contain a `namespace`, +which will be used by the program to identify the concrete Java class for the reflection process. + +The following example shows the `User` schema used previously. But this time it contains a namespace +pointing to the location(in this case the package), where the `User` class for the reflection could be found. + +```java +// avro schema with namespace +final String schema = + "{\"type\": \"record\", " + + "\"name\": \"User\", " + + "\"namespace\": \"org.apache.flink.formats.parquet.avro\", " + + "\"fields\": [\n" + + " {\"name\": \"name\", \"type\": \"string\" },\n" + + " {\"name\": \"favoriteNumber\", \"type\": [\"int\", \"null\"] },\n" + + " {\"name\": \"favoriteColor\", \"type\": [\"string\", \"null\"] }\n" + + " ]\n" + + " }"; + +``` + +Parquet files created with this schema will contain meta information like: -This example, similar to the bounded batch example, uses the same POJO Java class `Datum` -and monitors for the new files every second to read Avro Reflect records from Parquet files -infinitely as new files are added to the directory. +```text +creator: parquet-mr version 1.12.2 (build 77e30c8093386ec52c3cfa6c34b7ef3321322c94) +extra: parquet.avro.schema = +{"type":"record","name":"User","namespace":"org.apache.flink.formats.parquet.avro","fields":[{"name":"name","type":"string"},{"name":"favoriteNumber","type":["int","null"]},{"name":"favoriteColor","type":["string","null"]}]} +extra: writer.model.name = avro + +file schema: org.apache.flink.formats.parquet.avro.User +-------------------------------------------------------------------------------- +name: REQUIRED BINARY L:STRING R:0 D:0 +favoriteNumber: OPTIONAL INT32 R:0 D:1 +favoriteColor: OPTIONAL BINARY L:STRING R:0 D:1 + +row group 1: RC:3 TS:143 OFFSET:4 +-------------------------------------------------------------------------------- +name: BINARY UNCOMPRESSED DO:0 FPO:4 SZ:47/47/1.00 VC:3 ENC:PLAIN,BIT_PACKED ST:[min: Jack, max: Tom, num_nulls: 0] +favoriteNumber: INT32 UNCOMPRESSED DO:0 FPO:51 SZ:41/41/1.00 VC:3 ENC:RLE,PLAIN,BIT_PACKED ST:[min: 1, max: 3, num_nulls: 0] +favoriteColor: BINARY UNCOMPRESSED DO:0 FPO:92 SZ:55/55/1.00 VC:3 ENC:RLE,PLAIN,BIT_PACKED ST:[min: green, max: yellow, num_nulls: 0] + +``` + +With the `User` class defined in the package org.apache.flink.formats.parquet.avro: + +```java +public class User { + private String name; + private Integer favoriteNumber; + private String favoriteColor; + + public User() {} + + public User(String name, Integer favoriteNumber, String favoriteColor) { + this.name = name; + this.favoriteNumber = favoriteNumber; + this.favoriteColor = favoriteColor; + } + + public String getName() { + return name; + } + + public Integer getFavoriteNumber() { + return favoriteNumber; + } + + public String getFavoriteColor() { + return favoriteColor; + } + } + +``` + +you can write the following program to read Avro Reflect records of User type from parquet files: ```java final FileSource source = FileSource.forRecordStreamFormat( - AvroParquetReaders.forReflectRecord(Datum.class), /* Flink Path */) - .monitorContinuously(Duration.ofSeconds(1L)) + AvroParquetReaders.forReflectRecord(User.class), /* Flink Path */) .build(); final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.enableCheckpointing(10L); - + final DataStream stream = env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); ``` diff --git a/docs/content/docs/connectors/datastream/jdbc.md b/docs/content/docs/connectors/datastream/jdbc.md index bc96d23d07b0f..85c67235b0c06 100644 --- a/docs/content/docs/connectors/datastream/jdbc.md +++ b/docs/content/docs/connectors/datastream/jdbc.md @@ -73,7 +73,7 @@ JdbcExecutionOptions.builder() .withBatchIntervalMs(200) // optional: default = 0, meaning no time-based execution is done .withBatchSize(1000) // optional: default = 5000 values .withMaxRetries(5) // optional: default = 3 -.build() +.build(); ``` A JDBC batch is executed as soon as one of the following conditions is true: @@ -184,7 +184,7 @@ In such cases, please use the following API to construct `JdbcExactlyOnceOptions ```java JdbcExactlyOnceOptions.builder() .withTransactionPerConnection(true) -.build() +.build(); ``` This will make Flink use a separate connection for every XA transaction. This may require adjusting connection limits. For PostgreSQL and MySQL, this can be done by increasing `max_connections`. diff --git a/docs/content/docs/connectors/datastream/kafka.md b/docs/content/docs/connectors/datastream/kafka.md index ff48b0a37d3ed..8a1fbede6a02d 100644 --- a/docs/content/docs/connectors/datastream/kafka.md +++ b/docs/content/docs/connectors/datastream/kafka.md @@ -73,19 +73,19 @@ The following properties are **required** for building a KafkaSource: Kafka source provide 3 ways of topic-partition subscription: - Topic list, subscribing messages from all partitions in a list of topics. For example: ```java - KafkaSource.builder().setTopics("topic-a", "topic-b") + KafkaSource.builder().setTopics("topic-a", "topic-b"); ``` - Topic pattern, subscribing messages from all topics whose name matches the provided regular expression. For example: ```java - KafkaSource.builder().setTopicPattern("topic.*") + KafkaSource.builder().setTopicPattern("topic.*"); ``` - Partition set, subscribing partitions in the provided partition set. For example: ```java final HashSet partitionSet = new HashSet<>(Arrays.asList( new TopicPartition("topic-a", 0), // Partition 0 of topic "topic-a" new TopicPartition("topic-b", 5))); // Partition 5 of topic "topic-b" - KafkaSource.builder().setPartitions(partitionSet) + KafkaSource.builder().setPartitions(partitionSet); ``` ### Deserializer A deserializer is required for parsing Kafka messages. Deserializer (Deserialization schema) can be @@ -103,7 +103,7 @@ Kafka message value as string: import org.apache.kafka.common.serialization.StringDeserializer; KafkaSource.builder() - .setDeserializer(KafkaRecordDeserializationSchema.valueOnly(StringSerializer.class)); + .setDeserializer(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class)); ``` ### Starting Offset @@ -121,7 +121,7 @@ KafkaSource.builder() // Start from earliest offset .setStartingOffsets(OffsetsInitializer.earliest()) // Start from latest offset - .setStartingOffsets(OffsetsInitializer.latest()) + .setStartingOffsets(OffsetsInitializer.latest()); ``` You can also implement a custom offsets initializer if built-in initializers above cannot fulfill @@ -170,7 +170,7 @@ JAAS configuration: ```java KafkaSource.builder() .setProperty("sasl.mechanism", "PLAIN") - .setProperty("sasl.jaas.config", "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"username\" password=\"password\";") + .setProperty("sasl.jaas.config", "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"username\" password=\"password\";"); ``` ### Dynamic Partition Discovery @@ -180,7 +180,7 @@ topic-partition subscribing pattern. To enable partition discovery, set a non-ne property ```partition.discovery.interval.ms```: ```java KafkaSource.builder() - .setProperty("partition.discovery.interval.ms", "10000") // discover new partitions per 10 seconds + .setProperty("partition.discovery.interval.ms", "10000"); // discover new partitions per 10 seconds ``` {{< hint warning >}} Partition discovery is **disabled** by default. You need to explicitly set the partition discovery @@ -192,7 +192,7 @@ By default, the record will use the timestamp embedded in Kafka ```ConsumerRecor time. You can define your own ```WatermarkStrategy``` for extract event time from the record itself, and emit watermark downstream: ```java -env.fromSource(kafkaSource, new CustomWatermarkStrategy(), "Kafka Source With Custom Watermark Strategy") +env.fromSource(kafkaSource, new CustomWatermarkStrategy(), "Kafka Source With Custom Watermark Strategy"); ``` [This documentation]({{< ref "docs/dev/datastream/event-time/generating_watermarks.md" >}}) describes details about how to define a ```WatermarkStrategy```. @@ -361,7 +361,7 @@ Kafka sink provides a builder class to construct an instance of a KafkaSink. The shows how to write String records to a Kafka topic with a delivery guarantee of at least once. ```java -DataStream stream = ... +DataStream stream = ...; KafkaSink sink = KafkaSink.builder() .setBootstrapServers(brokers) diff --git a/docs/content/docs/connectors/datastream/pulsar.md b/docs/content/docs/connectors/datastream/pulsar.md index 2432227b180e9..15115c673b10b 100644 --- a/docs/content/docs/connectors/datastream/pulsar.md +++ b/docs/content/docs/connectors/datastream/pulsar.md @@ -24,14 +24,13 @@ under the License. # Apache Pulsar Connector -Flink provides an [Apache Pulsar](https://pulsar.apache.org) source connector for reading data from Pulsar topics with exactly-once guarantees. +Flink provides an [Apache Pulsar](https://pulsar.apache.org) connector for reading and writing data from and to Pulsar topics with exactly-once guarantees. ## Dependency -You can use the connector with Pulsar 2.7.0 or higher. However, the Pulsar source connector supports -Pulsar [transactions](https://pulsar.apache.org/docs/en/txn-what/), -it is recommended to use Pulsar 2.8.0 or higher releases. -For details on Pulsar compatibility, please refer to the [PIP-72](https://github.com/apache/pulsar/wiki/PIP-72%3A-Introduce-Pulsar-Interface-Taxonomy%3A-Audience-and-Stability-Classification). +You can use the connector with the Pulsar 2.8.1 or higher. Because the Pulsar connector supports +Pulsar [transactions](https://pulsar.apache.org/docs/en/txn-what/), it is recommended to use the Pulsar 2.9.2 or higher. +Details on Pulsar compatibility can be found in [PIP-72](https://github.com/apache/pulsar/wiki/PIP-72%3A-Introduce-Pulsar-Interface-Taxonomy%3A-Audience-and-Stability-Classification). {{< artifact flink-connector-pulsar >}} @@ -43,18 +42,16 @@ See how to link with them for cluster execution [here]({{< ref "docs/dev/configu {{< hint info >}} This part describes the Pulsar source based on the new [data source]({{< ref "docs/dev/datastream/sources.md" >}}) API. - -If you want to use the legacy `SourceFunction` or on Flink 1.13 or lower releases, just use the StreamNative's [pulsar-flink](https://github.com/streamnative/pulsar-flink). {{< /hint >}} ### Usage -Pulsar source provides a builder class for constructing an instance of PulsarSource. The code snippet below shows -how to build a PulsarSource to consume messages from the earliest cursor of topic "persistent://public/default/my-topic", -with **Exclusive** subscription `my-subscription` and deserialize the raw payload of the messages as strings. +The Pulsar source provides a builder class for constructing a PulsarSource instance. The code snippet below builds a PulsarSource instance. It consumes messages from the earliest cursor of the topic +"persistent://public/default/my-topic" in **Exclusive** subscription type (`my-subscription`) +and deserializes the raw payload of the messages as strings. ```java -PulsarSource pulsarSource = PulsarSource.builder() +PulsarSource source = PulsarSource.builder() .setServiceUrl(serviceUrl) .setAdminUrl(adminUrl) .setStartCursor(StartCursor.earliest()) @@ -69,13 +66,17 @@ env.fromSource(source, WatermarkStrategy.noWatermarks(), "Pulsar Source"); The following properties are **required** for building a PulsarSource: -- Pulsar service url, configured by `setServiceUrl(String)` -- Pulsar service http url (aka. admin url), configured by `setAdminUrl(String)` +- Pulsar service URL, configured by `setServiceUrl(String)` +- Pulsar service HTTP URL (also known as admin URL), configured by `setAdminUrl(String)` - Pulsar subscription name, configured by `setSubscriptionName(String)` - Topics / partitions to subscribe, see the following - [Topic-partition subscription](#topic-partition-subscription) for more details. + [topic-partition subscription](#topic-partition-subscription) for more details. - Deserializer to parse Pulsar messages, see the following - [Deserializer](#deserializer) for more details. + [deserializer](#deserializer) for more details. + +It is recommended to set the consumer name in Pulsar Source by `setConsumerName(String)`. +This sets a unique name for the Flink connector in the Pulsar statistic dashboard. +You can use it to monitor the performance of your Flink connector and applications. ### Topic-partition Subscription @@ -83,105 +84,121 @@ Pulsar source provide two ways of topic-partition subscription: - Topic list, subscribing messages from all partitions in a list of topics. For example: ```java - PulsarSource.builder().setTopics("some-topic1", "some-topic2") + PulsarSource.builder().setTopics("some-topic1", "some-topic2"); // Partition 0 and 2 of topic "topic-a" - PulsarSource.builder().setTopics("topic-a-partition-0", "topic-a-partition-2") + PulsarSource.builder().setTopics("topic-a-partition-0", "topic-a-partition-2"); ``` - Topic pattern, subscribing messages from all topics whose name matches the provided regular expression. For example: ```java - PulsarSource.builder().setTopicPattern("topic-*") + PulsarSource.builder().setTopicPattern("topic-*"); ``` #### Flexible Topic Naming -Since Pulsar 2.0, all topic names internally have the form `{persistent|non-persistent}://tenant/namespace/topic`. +Since Pulsar 2.0, all topic names internally are in a form of `{persistent|non-persistent}://tenant/namespace/topic`. Now, for partitioned topics, you can use short names in many cases (for the sake of simplicity). The flexible naming system stems from the fact that there is now a default topic type, tenant, and namespace in a Pulsar cluster. -Topic property | Default -:------------|:------- -topic type | `persistent` -tenant | `public` -namespace | `default` +| Topic property | Default | +|:---------------|:-------------| +| topic type | `persistent` | +| tenant | `public` | +| namespace | `default` | -This table lists a mapping relationship between your input topic name and translated topic name: +This table lists a mapping relationship between your input topic name and the translated topic name: -Input topic name | Translated topic name -:----------------|:--------------------- -`my-topic` | `persistent://public/default/my-topic` -`my-tenant/my-namespace/my-topic` | `persistent://my-tenant/my-namespace/my-topic` +| Input topic name | Translated topic name | +|:----------------------------------|:-----------------------------------------------| +| `my-topic` | `persistent://public/default/my-topic` | +| `my-tenant/my-namespace/my-topic` | `persistent://my-tenant/my-namespace/my-topic` | {{< hint warning >}} -For non-persistent topics, you need to continue to specify the entire topic name, +For non-persistent topics, you need to specify the entire topic name, as the default-based rules do not apply for non-partitioned topics. -Thus, you cannot use a short name like `non-persistent://my-topic` and would need to use `non-persistent://public/default/my-topic` instead. +Thus, you cannot use a short name like `non-persistent://my-topic` and need to use `non-persistent://public/default/my-topic` instead. {{< /hint >}} #### Subscribing Pulsar Topic Partition Internally, Pulsar divides a partitioned topic as a set of non-partitioned topics according to the partition size. -For example, if a `simple-string` topic with 3 partitions is created under the `sample` tenant with `flink` namespace. +For example, if a `simple-string` topic with 3 partitions is created under the `sample` tenant with the `flink` namespace. The topics on Pulsar would be: -Topic name | Partitioned -:--------- | :---------- -`persistent://sample/flink/simple-string` | Y -`persistent://sample/flink/simple-string-partition-0` | N -`persistent://sample/flink/simple-string-partition-1` | N -`persistent://sample/flink/simple-string-partition-2` | N +| Topic name | Partitioned | +|:------------------------------------------------------|:------------| +| `persistent://sample/flink/simple-string` | Y | +| `persistent://sample/flink/simple-string-partition-0` | N | +| `persistent://sample/flink/simple-string-partition-1` | N | +| `persistent://sample/flink/simple-string-partition-2` | N | You can directly consume messages from the topic partitions by using the non-partitioned topic names above. -For example, use `PulsarSource.builder().setTopics("sample/flink/simple-string-partition-1", "sample/flink/simple-string-partition-2")` would consume the partitions 1 and 2 of the `sample/flink/simple-string` topic. +For example, use `PulsarSource.builder().setTopics("sample/flink/simple-string-partition-1", "sample/flink/simple-string-partition-2")` +would consume the partitions 1 and 2 of the `sample/flink/simple-string` topic. -#### RegexSubscriptionMode for Topic Pattern +#### Setting Topic Patterns -Pulsar connector extracts the topic type (`persistent` or `non-persistent`) from the given topic pattern. -For example, `PulsarSource.builder().setTopicPattern("non-persistent://my-topic*")` would be `non-persistent`. -The topic type would be `persistent` if you do not provide the topic type in the regular expression. +The Pulsar source extracts the topic type (`persistent` or `non-persistent`) from the provided topic pattern. +For example, you can use the `PulsarSource.builder().setTopicPattern("non-persistent://my-topic*")` to specify a `non-persistent` topic. +By default, a `persistent` topic is created if you do not specify the topic type in the regular expression. -To consume both `persistent` and `non-persistent` topics based on the topic pattern, -you can use `setTopicPattern("topic-*", RegexSubscriptionMode.AllTopics)`. -Pulsar connector would filter the available topics by the `RegexSubscriptionMode`. +You can use `setTopicPattern("topic-*", RegexSubscriptionMode.AllTopics)` to consume +both `persistent` and `non-persistent` topics based on the topic pattern. +The Pulsar source would filter the available topics by the `RegexSubscriptionMode`. ### Deserializer -A deserializer (Deserialization schema) is required for parsing Pulsar messages. The deserializer is -configured by `setDeserializationSchema(PulsarDeserializationSchema)`. +A deserializer (`PulsarDeserializationSchema`) is for decoding Pulsar messages from bytes. +You can configure the deserializer using `setDeserializationSchema(PulsarDeserializationSchema)`. The `PulsarDeserializationSchema` defines how to deserialize a Pulsar `Message`. If only the raw payload of a message (message data in bytes) is needed, -you can use the predefined `PulsarDeserializationSchema`. Pulsar connector provides three types of implementation. +you can use the predefined `PulsarDeserializationSchema`. Pulsar connector provides three implementation methods. - Decode the message by using Pulsar's [Schema](https://pulsar.apache.org/docs/en/schema-understand/). ```java // Primitive types - PulsarDeserializationSchema.pulsarSchema(Schema) + PulsarDeserializationSchema.pulsarSchema(Schema); // Struct types (JSON, Protobuf, Avro, etc.) - PulsarDeserializationSchema.pulsarSchema(Schema, Class) + PulsarDeserializationSchema.pulsarSchema(Schema, Class); // KeyValue type - PulsarDeserializationSchema.pulsarSchema(Schema, Class, Class) + PulsarDeserializationSchema.pulsarSchema(Schema, Class, Class); ``` - Decode the message by using Flink's `DeserializationSchema` ```java - PulsarDeserializationSchema.flinkSchema(DeserializationSchema) + PulsarDeserializationSchema.flinkSchema(DeserializationSchema); ``` - Decode the message by using Flink's `TypeInformation` ```java - PulsarDeserializationSchema.flinkTypeInfo(TypeInformation, ExecutionConfig) + PulsarDeserializationSchema.flinkTypeInfo(TypeInformation, ExecutionConfig); ``` +If using KeyValue type or Struct types, the pulsar `Schema` does not contain type class info which is +needed by `PulsarSchemaTypeInformation`. So the two APIs provides 2 parameter to pass the type info. + +A example would be: + +```java + // Primitive types: do not need to provide type class info + PulsarDeserializationSchema.pulsarSchema(Schema.INT32); + + // Struct types (JSON, Protobuf, Avro, etc.) + PulsarDeserializationSchema.pulsarSchema(Schema.AVRO(SomeClass), SomeClass.class); + + // KeyValue type + PulsarDeserializationSchema.pulsarSchema(Schema.KeyValue(Schema.INT32, Schema.AVRO(SomeClass)), Integer.class, SomeClass.class); +``` Pulsar `Message` contains some [extra properties](https://pulsar.apache.org/docs/en/concepts-messaging/#messages), -such as message key, message publish time, message time, application defined key/value pairs that will be attached to the message, etc. -These properties could be acquired by the `Message` interface. +such as message key, message publish time, message time, and application-defined key/value pairs etc. +These properties could be defined in the `Message` interface. If you want to deserialize the Pulsar message by these properties, you need to implement `PulsarDeserializationSchema`. -And ensure that the `TypeInformation` from the `PulsarDeserializationSchema.getProducedType()` must be correct. -Flink would use this `TypeInformation` for passing the messages to downstream operators. +Ensure that the `TypeInformation` from the `PulsarDeserializationSchema.getProducedType()` is correct. +Flink uses this `TypeInformation` to pass the messages to downstream operators. ### Pulsar Subscriptions @@ -196,106 +213,108 @@ The subscription name is required for consuming messages. Pulsar connector suppo There is no difference between `Exclusive` and `Failover` in the Pulsar connector. When a Flink reader crashes, all (non-acknowledged and subsequent) messages are redelivered to the available Flink readers. -By default, if no subscription type is defined, Pulsar source uses `Shared` subscription. +By default, if no subscription type is defined, Pulsar source uses the `Shared` subscription type. ```java // Shared subscription with name "my-shared" -PulsarSource.builder().setSubscriptionName("my-shared") +PulsarSource.builder().setSubscriptionName("my-shared"); // Exclusive subscription with name "my-exclusive" -PulsarSource.builder().setSubscriptionName("my-exclusive").setSubscriptionType(SubscriptionType.Exclusive) +PulsarSource.builder().setSubscriptionName("my-exclusive").setSubscriptionType(SubscriptionType.Exclusive); ``` -If you want to use `Key_Shared` subscription type on the Pulsar connector. Ensure that you provide a `RangeGenerator` implementation. -The `RangeGenerator` generates a set of key hash ranges so that -a respective reader subtask will only dispatch messages where the hash of the message key is contained in the specified range. +Ensure that you provide a `RangeGenerator` implementation if you want to use the `Key_Shared` subscription type on the Pulsar connector. +The `RangeGenerator` generates a set of key hash ranges so that a respective reader subtask only dispatches messages where the hash of the message key is contained in the specified range. -Pulsar connector would use a `UniformRangeGenerator` which would divides the range by the Flink source parallelism -if no `RangeGenerator` is provided in the `Key_Shared` subscription type. +The Pulsar connector uses `UniformRangeGenerator` that divides the range by the Flink source +parallelism if no `RangeGenerator` is provided in the `Key_Shared` subscription type. ### Starting Position -Pulsar source is able to consume messages starting from different positions by `setStartCursor(StartCursor)`. +The Pulsar source is able to consume messages starting from different positions by setting the `setStartCursor(StartCursor)` option. Built-in start cursors include: - Start from the earliest available message in the topic. ```java - StartCursor.earliest() + StartCursor.earliest(); ``` - Start from the latest available message in the topic. ```java - StartCursor.latest() + StartCursor.latest(); ``` - Start from a specified message between the earliest and the latest. - Pulsar connector would consume from the latest available message if the message id doesn't exist. +The Pulsar connector consumes from the latest available message if the message ID does not exist. The start message is included in consuming result. ```java - StartCursor.fromMessageId(MessageId) + StartCursor.fromMessageId(MessageId); ``` - Start from a specified message between the earliest and the latest. - Pulsar connector would consume from the latest available message if the message id doesn't exist. +The Pulsar connector consumes from the latest available message if the message ID doesn't exist. Include or exclude the start message by using the second boolean parameter. ```java - StartCursor.fromMessageId(MessageId, boolean) + StartCursor.fromMessageId(MessageId, boolean); ``` -- Start from the specified message time by `Message.getEventTime()`. +- Start from the specified message time by `Message.getPublishTime()`. ```java - StartCursor.fromMessageTime(long) + StartCursor.fromPublishTime(long); ``` {{< hint info >}} Each Pulsar message belongs to an ordered sequence on its topic. The sequence ID (`MessageId`) of the message is ordered in that sequence. -`MessageId` contains some extra information (the ledger, entry, partition) on how the message is stored, +The `MessageId` contains some extra information (the ledger, entry, partition) about how the message is stored, you can create a `MessageId` by using `DefaultImplementation.newMessageId(long ledgerId, long entryId, int partitionIndex)`. {{< /hint >}} ### Boundedness -Pulsar source supports streaming and batch running modes. -By default, the `PulsarSource` is set to run in the streaming mode. +The Pulsar source supports streaming and batch execution mode. +By default, the `PulsarSource` is configured for unbounded data. -In streaming mode, Pulsar source never stops until a Flink job fails or is cancelled. However, -you can set Pulsar source stopping at a stop position by using ```setUnboundedStopCursor(StopCursor)```. -The Pulsar source will finish when all partitions reach their specified stop positions. +For unbounded data the Pulsar source never stops until a Flink job is stopped or failed. +You can use the `setUnboundedStopCursor(StopCursor)` to set the Pulsar source to stop at a specific stop position. -You can use ```setBoundedStopCursor(StopCursor)``` to specify a stop position so that the Pulsar source can run in the batch mode. -When all partitions have reached their stop positions, the source will finish. +You can use `setBoundedStopCursor(StopCursor)` to specify a stop position for bounded data. Built-in stop cursors include: -- Connector will never stop consuming. +- The Pulsar source never stops consuming messages. ```java - StopCursor.never() + StopCursor.never(); ``` -- Stop at the latest available message in Pulsar when the connector starts consuming. +- Stop at the latest available message when the Pulsar source starts consuming messages. ```java - StopCursor.latest() + StopCursor.latest(); ``` -- Stop when connector meet a given message, or stop at a message which is produced after this given message. +- Stop when the connector meets a given message, or stop at a message which is produced after this given message. ```java - StopCursor.atMessageId(MessageId) + StopCursor.atMessageId(MessageId); ``` -- Stop but include the given message in consuming result. +- Stop but include the given message in the consuming result. ```java - StopCursor.afterMessageId(MessageId) + StopCursor.afterMessageId(MessageId); ``` -- Stop at the specified message time by `Message.getEventTime()`. +- Stop at the specified message time by `Message.getPublishTime()`. ```java - StopCursor.atEventTime(long) + StopCursor.atPublishTime(long); ``` -### Configurable Options +{{< hint warning >}} +StopCursor.atEventTime(long) is now deprecated. + {{< /hint >}} + +### Source Configurable Options In addition to configuration options described above, you can set arbitrary options for `PulsarClient`, -`PulsarAdmin`, Pulsar `Consumer` and `PulsarSource` by using `setConfig(ConfigOption, T)` and `setConfig(Configuration)`. +`PulsarAdmin`, Pulsar `Consumer` and `PulsarSource` by using `setConfig(ConfigOption, T)`, +`setConfig(Configuration)` and `setConfig(Properties)`. #### PulsarClient Options -Pulsar connector use the [client API](https://pulsar.apache.org/docs/en/client-libraries-java/) -to create the `Consumer` instance. Pulsar connector extracts most parts of Pulsar's `ClientConfigurationData`, +The Pulsar connector uses the [client API](https://pulsar.apache.org/docs/en/client-libraries-java/) +to create the `Consumer` instance. The Pulsar connector extracts most parts of Pulsar's `ClientConfigurationData`, which is required for creating a `PulsarClient`, as Flink configuration options in `PulsarOptions`. {{< generated/pulsar_client_configuration >}} @@ -303,8 +322,9 @@ which is required for creating a `PulsarClient`, as Flink configuration options #### PulsarAdmin Options The [admin API](https://pulsar.apache.org/docs/en/admin-api-overview/) is used for querying topic metadata -and for discovering the desired topics when Pulsar connector uses topic pattern subscription. It would share most part of the -configuration options with the client API. The configuration options listed here are only used in the admin API. +and for discovering the desired topics when the Pulsar connector uses topic-pattern subscription. +It shares most part of the configuration options with the client API. +The configuration options listed here are only used in the admin API. They are also defined in `PulsarOptions`. {{< generated/pulsar_admin_configuration >}} @@ -312,34 +332,34 @@ They are also defined in `PulsarOptions`. #### Pulsar Consumer Options In general, Pulsar provides the Reader API and Consumer API for consuming messages in different scenarios. -Flink's Pulsar connector uses the Consumer API. It extracts most parts of Pulsar's `ConsumerConfigurationData` as Flink configuration options in `PulsarSourceOptions`. +The Pulsar connector uses the Consumer API. It extracts most parts of Pulsar's `ConsumerConfigurationData` as Flink configuration options in `PulsarSourceOptions`. {{< generated/pulsar_consumer_configuration >}} #### PulsarSource Options The configuration options below are mainly used for customizing the performance and message acknowledgement behavior. -You can just leave them alone if you do not meet any performance issues. +You can ignore them if you do not have any performance issues. {{< generated/pulsar_source_configuration >}} ### Dynamic Partition Discovery To handle scenarios like topic scaling-out or topic creation without restarting the Flink -job, Pulsar source can be configured to periodically discover new partitions under provided -topic-partition subscribing pattern. To enable partition discovery, set a non-negative value for -the option `PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS`: +job, the Pulsar source periodically discover new partitions under a provided +topic-partition subscription pattern. To enable partition discovery, you can set a non-negative value for +the `PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS` option: ```java // discover new partitions per 10 seconds PulsarSource.builder() - .setConfig(PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS, 10000); + .setConfig(PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS, 10000); ``` {{< hint warning >}} -- Partition discovery is **enabled** by default. Pulsar connector would query the topic metadata every 30 seconds. -- You need to set the partition discovery interval to a negative value to disable this feature. -- The partition discovery would be disabled in batch mode even if you set this option with a non-negative value. +- Partition discovery is **enabled** by default. The Pulsar connector queries the topic metadata every 30 seconds. +- To disable partition discovery, you need to set a negative partition discovery interval. +- Partition discovery is disabled for bounded data even if you set this option with a non-negative value. {{< /hint >}} ### Event Time and Watermarks @@ -349,7 +369,7 @@ You can define your own `WatermarkStrategy` to extract the event time from the m and emit the watermark downstream: ```java -env.fromSource(pulsarSource, new CustomWatermarkStrategy(), "Pulsar Source With Custom Watermark Strategy") +env.fromSource(pulsarSource, new CustomWatermarkStrategy(), "Pulsar Source With Custom Watermark Strategy"); ``` [This documentation]({{< ref "docs/dev/datastream/event-time/generating_watermarks.md" >}}) describes @@ -359,7 +379,7 @@ details about how to define a `WatermarkStrategy`. When a subscription is created, Pulsar [retains](https://pulsar.apache.org/docs/en/concepts-architecture-overview/#persistent-storage) all messages, even if the consumer is disconnected. The retained messages are discarded only when the connector acknowledges that all these messages are processed successfully. -Pulsar connector supports four subscription types, which makes the acknowledgement behaviors variety among different subscriptions. +The Pulsar connector supports four subscription types, which makes the acknowledgement behaviors vary among different subscriptions. #### Acknowledgement on Exclusive and Failover Subscription Types @@ -367,18 +387,18 @@ Pulsar connector supports four subscription types, which makes the acknowledgeme the latest successfully consumed message. All the message before the given message are marked with a consumed status. -Pulsar source acknowledges the current consuming message when checkpoints are **completed**, -to ensure the consistency between Flink's checkpoint state and committed position on Pulsar brokers. +The Pulsar source acknowledges the current consuming message when checkpoints are **completed**, +to ensure the consistency between Flink's checkpoint state and committed position on the Pulsar brokers. If checkpointing is disabled, Pulsar source periodically acknowledges messages. -You can set the acknowledgement period by using the `PulsarSourceOptions.PULSAR_AUTO_COMMIT_CURSOR_INTERVAL` option. +You can use the `PulsarSourceOptions.PULSAR_AUTO_COMMIT_CURSOR_INTERVAL` option to set the acknowledgement period. Pulsar source does **NOT** rely on committed positions for fault tolerance. -Acknowledging messages is only for exposing the progress of consumer and monitoring on these two subscription types. +Acknowledging messages is only for exposing the progress of consumers and monitoring on these two subscription types. #### Acknowledgement on Shared and Key_Shared Subscription Types -`Shared` and `Key_Shared` subscription types need to acknowledge messages one by one. You can acknowledge +In `Shared` and `Key_Shared` subscription types, messages are acknowledged one by one. You can acknowledge a message in a transaction and commit it to Pulsar. You should enable transaction in the Pulsar `borker.conf` file when using these two subscription types in connector: @@ -387,19 +407,411 @@ You should enable transaction in the Pulsar `borker.conf` file when using these transactionCoordinatorEnabled=true ``` -Pulsar transaction would be created with 3 hours as the timeout by default. Make sure that timeout > checkpoint interval + maximum recovery time. -A shorter checkpoint interval would increase the consuming performance. -You can change the transaction timeout by using the `PulsarSourceOptions.PULSAR_TRANSACTION_TIMEOUT_MILLIS` option. +The default timeout for Pulsar transactions is 3 hours. +Make sure that that timeout is greater than checkpoint interval + maximum recovery time. +A shorter checkpoint interval indicates a better consuming performance. +You can use the `PulsarSourceOptions.PULSAR_TRANSACTION_TIMEOUT_MILLIS` option to change the transaction timeout. If checkpointing is disabled or you can not enable the transaction on Pulsar broker, you should set `PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE` to `true`. -The message would be immediately acknowledged after consuming. -We can not promise consistency in this scenario. +The message is immediately acknowledged after consuming. +No consistency guarantees can be made in this scenario. {{< hint info >}} All acknowledgements in a transaction are recorded in the Pulsar broker side. {{< /hint >}} +## Pulsar Sink + +The Pulsar Sink supports writing records into one or more Pulsar topics or a specified list of Pulsar partitions. + +{{< hint info >}} +This part describes the Pulsar sink based on the new +[data sink](https://cwiki.apache.org/confluence/display/FLINK/FLIP-191%3A+Extend+unified+Sink+interface+to+support+small+file+compaction) API. + +If you still want to use the legacy `SinkFunction` or on Flink 1.14 or previous releases, just use the StreamNative's +[pulsar-flink](https://github.com/streamnative/pulsar-flink). +{{< /hint >}} + +### Usage + +The Pulsar Sink uses a builder class to construct the `PulsarSink` instance. +This example writes a String record to a Pulsar topic with at-least-once delivery guarantee. + +```java +DataStream stream = ... + +PulsarSink sink = PulsarSink.builder() + .setServiceUrl(serviceUrl) + .setAdminUrl(adminUrl) + .setTopics("topic1") + .setSerializationSchema(PulsarSerializationSchema.flinkSchema(new SimpleStringSchema())) + .setDeliverGuarantee(DeliveryGuarantee.AT_LEAST_ONCE) + .build(); + +stream.sinkTo(sink); +``` + +The following properties are **required** for building PulsarSink: + +- Pulsar service url, configured by `setServiceUrl(String)` +- Pulsar service http url (aka. admin url), configured by `setAdminUrl(String)` +- Topics / partitions to write, see [writing targets](#writing-targets) for more details. +- Serializer to generate Pulsar messages, see [serializer](#serializer) for more details. + +It is recommended to set the producer name in Pulsar Source by `setProducerName(String)`. +This sets a unique name for the Flink connector in the Pulsar statistic dashboard. +You can use it to monitor the performance of your Flink connector and applications. + +### Producing to topics + +Defining the topics for producing is similar to the [topic-partition subscription](#topic-partition-subscription) +in the Pulsar source. We support a mix-in style of topic setting. You can provide a list of topics, +partitions, or both of them. + +```java +// Topic "some-topic1" and "some-topic2" +PulsarSink.builder().setTopics("some-topic1", "some-topic2") + +// Partition 0 and 2 of topic "topic-a" +PulsarSink.builder().setTopics("topic-a-partition-0", "topic-a-partition-2") + +// Partition 0 and 2 of topic "topic-a" and topic "some-topic2" +PulsarSink.builder().setTopics("topic-a-partition-0", "topic-a-partition-2", "some-topic2") +``` + +The topics you provide support auto partition discovery. We query the topic metadata from the Pulsar in a fixed interval. +You can use the `PulsarSinkOptions.PULSAR_TOPIC_METADATA_REFRESH_INTERVAL` option to change the discovery interval option. + +Configuring writing targets can be replaced by using a custom [`TopicRouter`] +[message routing](#message-routing). Configuring partitions on the Pulsar connector is explained in the [flexible topic naming](#flexible-topic-naming) section. + +{{< hint warning >}} +If you build the Pulsar sink based on both the topic and its corresponding partitions, Pulsar sink merges them and only uses the topic. + +For example, when using the `PulsarSink.builder().setTopics("some-topic1", "some-topic1-partition-0")` option to build the Pulsar sink, +this is simplified to `PulsarSink.builder().setTopics("some-topic1")`. +{{< /hint >}} + +#### Dynamic Topics by income messages + +Topics could be defined by the message content instead of providing the fix topic set. You can dynamically +provide the topic by implementing `TopicExtractor`. The topic metadata in `TopicExtractor` can be queried +by using `TopicMetadataProvider` and the query result would be expired after we have queried for +`PulsarSinkOptions.PULSAR_TOPIC_METADATA_REFRESH_INTERVAL` time. + +You can return two types of value in `TopicExtractor`. A topic name with or without partition information. + +1. If you don't want to provide the partition, we would query the partition size and passing all the partitions to `TopicRouter`. The partition size would be cached in `PulsarSinkOptions.PULSAR_TOPIC_METADATA_REFRESH_INTERVAL`. +2. If you provided the topic partition, we would do nothing but just pass it to downstream. + +### Serializer + +A serializer (`PulsarSerializationSchema`) is required for serializing the record instance into bytes. +Similar to `PulsarSource`, Pulsar sink supports both Flink's `SerializationSchema` and +Pulsar's `Schema`. Pulsar's `Schema.AUTO_PRODUCE_BYTES()` is not supported in the Pulsar sink. + +If you do not need the message key and other message properties in Pulsar's +[Message](https://pulsar.apache.org/api/client/2.9.0-SNAPSHOT/org/apache/pulsar/client/api/Message.html) interface, +you can use the predefined `PulsarSerializationSchema`. The Pulsar sink provides two implementation methods. + +- Encode the message by using Pulsar's [Schema](https://pulsar.apache.org/docs/en/schema-understand/). + ```java + // Primitive types + PulsarSerializationSchema.pulsarSchema(Schema) + + // Struct types (JSON, Protobuf, Avro, etc.) + PulsarSerializationSchema.pulsarSchema(Schema, Class) + + // KeyValue type + PulsarSerializationSchema.pulsarSchema(Schema, Class, Class) + ``` +- Encode the message by using Flink's `SerializationSchema` + ```java + PulsarSerializationSchema.flinkSchema(SerializationSchema) + ``` + +[Schema evolution](https://pulsar.apache.org/docs/en/schema-evolution-compatibility/#schema-evolution) +can be enabled by users using `PulsarSerializationSchema.pulsarSchema()` and +`PulsarSinkBuilder.enableSchemaEvolution()`. This means that any broker schema validation is in place. + +```java +Schema schema = Schema.AVRO(SomePojo.class); +PulsarSerializationSchema pulsarSchema = PulsarSerializationSchema.pulsarSchema(schema, SomePojo.class); + +PulsarSink sink = PulsarSink.builder() + ... + .setSerializationSchema(pulsarSchema) + .enableSchemaEvolution() + .build(); +``` + +{{< hint warning >}} +If you use Pulsar schema without enabling schema evolution, the target topic will have a `Schema.BYTES` schema. +Consumers will need to handle the deserialization (if needed) themselves. + +For example, if you set `PulsarSerializationSchema.pulsarSchema(Schema.STRING)` without enabling schema evolution, +the schema stored in Pulsar topics is `Schema.BYTES`. +{{< /hint >}} + +### Message Routing + +Routing in Pulsar Sink is operated on the partition level. For a list of partitioned topics, +the routing algorithm first collects all partitions from different topics, and then calculates routing within all the partitions. +By default Pulsar Sink supports two router implementation. + +- `KeyHashTopicRouter`: use the hashcode of the message's key to decide the topic partition that messages are sent to. + + The message key is provided by `PulsarSerializationSchema.key(IN, PulsarSinkContext)` + You need to implement this interface and extract the message key when you want to send the message with the same key to the same topic partition. + + If you do not provide the message key. A topic partition is randomly chosen from the topic list. + + The message key can be hashed in two ways: `MessageKeyHash.JAVA_HASH` and `MessageKeyHash.MURMUR3_32_HASH`. + You can use the `PulsarSinkOptions.PULSAR_MESSAGE_KEY_HASH` option to choose the hash method. + +- `RoundRobinRouter`: Round-robin among all the partitions. + + All messages are sent to the first partition, and switch to the next partition after sending + a fixed number of messages. The batch size can be customized by the `PulsarSinkOptions.PULSAR_BATCHING_MAX_MESSAGES` option. + +Let’s assume there are ten messages and two topics. Topic A has two partitions while topic B has three partitions. +The batch size is set to five messages. In this case, topic A has 5 messages per partition which topic B does not receive any messages. + +You can configure custom routers by using the `TopicRouter` interface. +If you implement a `TopicRouter`, ensure that it is serializable. +And you can return partitions which are not available in the pre-discovered partition list. + +Thus, you do not need to specify topics using the `PulsarSinkBuilder.setTopics` option when you implement the custom topic router. + +```java +@PublicEvolving +public interface TopicRouter extends Serializable { + + String route(IN in, List partitions, PulsarSinkContext context); + + default void open(SinkConfiguration sinkConfiguration) { + // Nothing to do by default. + } +} +``` + +{{< hint info >}} +Internally, a Pulsar partition is implemented as a topic. The Pulsar client provides APIs to hide this +implementation detail and handles routing under the hood automatically. Pulsar Sink uses a lower client +API to implement its own routing layer to support multiple topics routing. + +For details, see [partitioned topics](https://pulsar.apache.org/docs/en/cookbooks-partitioned/). +{{< /hint >}} + +### Delivery Guarantee + +`PulsarSink` supports three delivery guarantee semantics. + +- `NONE`: Data loss can happen even when the pipeline is running. + Basically, we use a fire-and-forget strategy to send records to Pulsar topics in this mode. + It means that this mode has the highest throughput. +- `AT_LEAST_ONCE`: No data loss happens, but data duplication can happen after a restart from checkpoint. +- `EXACTLY_ONCE`: No data loss happens. Each record is sent to the Pulsar broker only once. + Pulsar Sink uses [Pulsar transaction](https://pulsar.apache.org/docs/en/transactions/) + and two-phase commit (2PC) to ensure records are sent only once even after pipeline restarts. + +### Delayed message delivery + +[Delayed message delivery](https://pulsar.apache.org/docs/en/next/concepts-messaging/#delayed-message-delivery) +enables you to delay the possibility to consume a message. With delayed message enabled, the Pulsar sink sends a message to the Pulsar topic +**immediately**, but the message is delivered to a consumer once the specified delay is over. + +Delayed message delivery only works in the `Shared` subscription type. In `Exclusive` and `Failover` +subscription types, the delayed message is dispatched immediately. + +You can configure the `MessageDelayer` to define when to send the message to the consumer. +The default option is to never delay the message dispatching. You can use the `MessageDelayer.fixed(Duration)` option to +Configure delaying all messages in a fixed duration. You can also implement the `MessageDelayer` +interface to dispatch messages at different time. + +{{< hint warning >}} +The dispatch time should be calculated by the `PulsarSinkContext.processTime()`. +{{< /hint >}} + +### Sink Configurable Options + +You can set options for `PulsarClient`, `PulsarAdmin`, Pulsar `Producer` and `PulsarSink` +by using `setConfig(ConfigOption, T)`, `setConfig(Configuration)` and `setConfig(Properties)`. + +#### PulsarClient and PulsarAdmin Options + +For details, refer to [PulsarAdmin options](#pulsaradmin-options). + +#### Pulsar Producer Options + +The Pulsar connector uses the Producer API to send messages. It extracts most parts of +Pulsar's `ProducerConfigurationData` as Flink configuration options in `PulsarSinkOptions`. + +{{< generated/pulsar_producer_configuration >}} + +#### PulsarSink Options + +The configuration options below are mainly used for customizing the performance and message +sending behavior. You can just leave them alone if you do not have any performance issues. + +{{< generated/pulsar_sink_configuration >}} + +### Sink Metrics + +This table lists supported metrics. +The first 6 metrics are standard Pulsar Sink metrics as described in +[FLIP-33: Standardize Connector Metrics]([https://cwiki.apache.org/confluence/display/FLINK/FLIP-33%3A+Standardize+Connector+Metrics](https://cwiki.apache.org/confluence/display/FLINK/FLIP-33%3A+Standardize+Connector+Metrics)) + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
ScopeMetricsUser VariablesDescriptionType
OperatornumBytesOutn/aThe total number of output bytes since the sink starts. Count towards the numBytesOut in TaskIOMetricsGroup.Counter
numBytesOutPerSecondn/aThe output bytes per secondMeter
numRecordsOutn/aThe total number of output records since the sink starts.Counter
numRecordsOutPerSecondn/aThe output records per secondMeter
numRecordsOutErrorsn/aThe total number of records failed to sendCounter
currentSendTimen/aThe time it takes to send the last record, from enqueue the message in client buffer to its ack.Gauge
PulsarSink.numAcksReceivedn/aThe number of acks received for sent messages.Counter
PulsarSink.sendLatencyMaxn/aThe maximum send latency in the last refresh interval across all producers.Gauge
PulsarSink.producer."ProducerName".sendLatency50PctProducerNameThe 50th percentile of send latency in the last refresh interval for a specific producer.Gauge
PulsarSink.producer."ProducerName".sendLatency75PctProducerNameThe 75th percentile of send latency in the last refresh interval for a specific producer.Gauge
PulsarSink.producer."ProducerName".sendLatency95PctProducerNameThe 95th percentile of send latency in the last refresh interval for a specific producer.Gauge
PulsarSink.producer."ProducerName".sendLatency99PctProducerNameThe 99th percentile of send latency in the last refresh interval for a specific producer.Gauge
PulsarSink.producer."ProducerName".sendLatency999PctProducerNameThe 99.9th percentile of send latency in the last refresh interval for a specific producer.Gauge
+ +{{< hint info >}} +- `numBytesOut`, `numRecordsOut`, `numRecordsOutErrors` are retrieved from Pulsar client metrics. + +- `currentSendTime` tracks the time from when the producer calls `sendAync()` to + the time when the message is acknowledged by the broker. This metric is not available in `NONE` delivery guarantee. +{{< /hint >}} + +The Pulsar producer refreshes its stats every 60 seconds by default. The PulsarSink retrieves the Pulsar producer +stats every 500ms. That means that `numRecordsOut`, `numBytesOut`, `numAcksReceived`, and `numRecordsOutErrors` +are updated every 60 seconds. To increase the metrics refresh frequency, you can change +the Pulsar producer stats refresh interval to a smaller value (minimum 1 second), as shown below. + +```java +builder.setConfig(PulsarOptions.PULSAR_STATS_INTERVAL_SECONDS. 1L) +``` + +`numBytesOutRate` and `numRecordsOutRate` are calculated based on the `numBytesOut` and `numRecordsOUt` +counter respectively. Flink internally uses a fixed 60 seconds window to calculate the rates. + +### Brief Design Rationale + +Pulsar sink follow the Sink API defined in +[FLIP-191](https://cwiki.apache.org/confluence/display/FLINK/FLIP-191%3A+Extend+unified+Sink+interface+to+support+small+file+compaction). + +#### Stateless SinkWriter + +In `EXACTLY_ONCE` mode, the Pulsar sink does not store transaction information in a checkpoint. +That means that new transactions will be created after a restart. +Therefore, any message in previous pending transactions is either aborted or timed out +(They are never visible to the downstream Pulsar consumer). +The Pulsar team is working to optimize the needed resources by unfinished pending transactions. + +#### Pulsar Schema Evolution + +[Pulsar Schema Evolution](https://pulsar.apache.org/docs/en/schema-evolution-compatibility/) allows +you to reuse the same Flink job after certain "allowed" data model changes, like adding or deleting +a field in a AVRO-based Pojo class. Please note that you can specify Pulsar schema validation rules +and define an auto schema update. For details, refer to [Pulsar Schema Evolution](https://pulsar.apache.org/docs/en/schema-evolution-compatibility/). + +## Known Issues + +This section describes some known issues about the Pulsar connectors. + +### Unstable on Java 11 + +Pulsar connector has some known issues on Java 11. It is recommended to run Pulsar connector +on Java 8. + +### No TransactionCoordinatorNotFound, but automatic reconnect + +Pulsar transactions are still in active development and are not stable. Pulsar 2.9.2 +introduces [a break change](https://github.com/apache/pulsar/pull/13135) in transactions. +If you use Pulsar 2.9.2 or higher with an older Pulsar client, you might get a `TransactionCoordinatorNotFound` exception. + +You can use the latest `pulsar-client-all` release to resolve this issue. + ## Upgrading to the Latest Connector Version The generic upgrade steps are outlined in [upgrading jobs and Flink versions guide]({{< ref "docs/ops/upgrading" >}}). @@ -407,7 +819,7 @@ The Pulsar connector does not store any state on the Flink side. The Pulsar conn For Pulsar, you additionally need to know these limitations: * Do not upgrade the Pulsar connector and Pulsar broker version at the same time. -* Always use a newer Pulsar client with Pulsar connector for consuming message from Pulsar. +* Always use a newer Pulsar client with Pulsar connector to consume messages from Pulsar. ## Troubleshooting @@ -417,13 +829,4 @@ If you have a problem with Pulsar when using Flink, keep in mind that Flink only and your problem might be independent of Flink and sometimes can be solved by upgrading Pulsar brokers, reconfiguring Pulsar brokers or reconfiguring Pulsar connector in Flink. -### Messages can be delayed on low volume topics - -When the Pulsar source connector reads from a low volume topic, users might observe a 10 seconds delay between messages. Pulsar buffers messages from topics by default. Before emitting to downstream -operators, the number of buffered records must be equal or larger than `PulsarSourceOptions.PULSAR_MAX_FETCH_RECORDS`. If the data volume is low, it could be that filling up the number of buffered records takes longer than `PULSAR_MAX_FETCH_TIME` (default to 10 seconds). If that's the case, it means that only after this time has passed the messages will be emitted. - -To avoid this behaviour, you need to change either the buffered records or the waiting time. - - - {{< top >}} diff --git a/docs/content/docs/connectors/table/elasticsearch.md b/docs/content/docs/connectors/table/elasticsearch.md index 78d0ce92c8ad4..93a50ddbcec65 100644 --- a/docs/content/docs/connectors/table/elasticsearch.md +++ b/docs/content/docs/connectors/table/elasticsearch.md @@ -140,12 +140,30 @@ Connector Options Password used to connect to Elasticsearch instance. If username is configured, this option must be configured with non-empty string as well. -
sink.delivery-guarantee
+
failure-handler
optional - no - NONE + yes + fail String - Optional delivery guarantee when committing. Valid values are NONE or AT_LEAST_ONCE. + Failure handling strategy in case a request to Elasticsearch fails. Valid strategies are: +
    +
  • fail: throws an exception if a request fails and thus causes a job failure.
  • +
  • ignore: ignores failures and drops the request.
  • +
  • retry-rejected: re-adds requests that have failed due to queue capacity saturation.
  • +
  • custom class name: for failure handling with a ActionRequestFailureHandler subclass.
  • +
+ + + +
sink.flush-on-checkpoint
+ optional + + true + Boolean + Flush on checkpoint or not. When disabled, a sink will not wait for all pending action requests + to be acknowledged by Elasticsearch on checkpoints. Thus, a sink does NOT provide any strong + guarantees for at-least-once delivery of action requests. +
sink.bulk-flush.max-actions
@@ -182,11 +200,11 @@ Connector Options
sink.bulk-flush.backoff.strategy
optional yes - NONE + DISABLED String Specify how to perform retries if any flush actions failed due to a temporary request error. Valid strategies are:
    -
  • NONE: no retry performed, i.e. fail after the first request error.
  • +
  • DISABLED: no retry performed, i.e. fail after the first request error.
  • CONSTANT: wait for backoff delay between retries.
  • EXPONENTIAL: initially wait for backoff delay and increase exponentially between retries.
@@ -208,14 +226,6 @@ Connector Options Duration Delay between each backoff attempt. For CONSTANT backoff, this is simply the delay between each retry. For EXPONENTIAL backoff, this is the initial base delay. - -
sink.parallelism
- optional - no - (none) - Integer - Defines the parallelism of the Elasticsearch sink operator. By default, the parallelism is determined by the framework using the same parallelism of the upstream chained operator. -
connection.path-prefix
optional diff --git a/docs/content/docs/connectors/table/hive/hive_catalog.md b/docs/content/docs/connectors/table/hive/hive_catalog.md index 90c1aebf0576f..932e18fcc0d36 100644 --- a/docs/content/docs/connectors/table/hive/hive_catalog.md +++ b/docs/content/docs/connectors/table/hive/hive_catalog.md @@ -137,28 +137,23 @@ Time taken: 0.028 seconds, Fetched: 0 row(s) ``` -#### step 2: configure Flink cluster and SQL CLI - -Add all Hive dependencies to `/lib` dir in Flink distribution, and modify SQL CLI's yaml config file `sql-cli-defaults.yaml` as following: - -```yaml - -execution: - type: streaming - ... - current-catalog: myhive # set the HiveCatalog as the current catalog of the session - current-database: mydatabase - -catalogs: - - name: myhive - type: hive - hive-conf-dir: /opt/hive-conf # contains hive-site.xml +#### step 2: start SQL Client, and create a Hive catalog with Flink SQL DDL + +Add all Hive dependencies to `/lib` dir in Flink distribution, and create a Hive catalog in Flink SQL CLI as following: + +```bash + +Flink SQL> CREATE CATALOG myhive WITH ( + 'type' = 'hive', + 'hive-conf-dir' = '/opt/hive-conf' +); + ``` #### step 3: set up a Kafka cluster -Bootstrap a local Kafka 2.3.0 cluster with a topic named "test", and produce some simple data to the topic as tuple of name and age. +Bootstrap a local Kafka cluster with a topic named "test", and produce some simple data to the topic as tuple of name and age. ```bash @@ -180,11 +175,12 @@ john,21 ``` -#### step 4: start SQL Client, and create a Kafka table with Flink SQL DDL +#### step 4: create a Kafka table with Flink SQL DDL -Start Flink SQL Client, create a simple Kafka 2.3.0 table via DDL, and verify its schema. +Create a simple Kafka table with Flink SQL DDL, and verify its schema. ```bash +Flink SQL> USE CATALOG myhive; Flink SQL> CREATE TABLE mykafka (name String, age Int) WITH ( 'connector.type' = 'kafka', diff --git a/docs/content/docs/connectors/table/hive/hive_dialect.md b/docs/content/docs/connectors/table/hive/hive_dialect.md index 38c0845b2e624..f8d2e675cbbc5 100644 --- a/docs/content/docs/connectors/table/hive/hive_dialect.md +++ b/docs/content/docs/connectors/table/hive/hive_dialect.md @@ -430,3 +430,5 @@ The following are some precautions for using the Hive dialect. location is only supported in Hive-2.4.0 or later. - Use [HiveModule]({{< ref "docs/connectors/table/hive/hive_functions" >}}#use-hive-built-in-functions-via-hivemodule) to run DML and DQL. +- Since Flink 1.15 you need to swap flink-table-planner-loader located in /lib with flink-table-planner_2.12 located in /opt to avoid the following exception. Please see [FLINK-25128](https://issues.apache.org/jira/browse/FLINK-25128) for more details. + {{error}} diff --git a/docs/content/docs/connectors/table/pulsar.md b/docs/content/docs/connectors/table/pulsar.md new file mode 100644 index 0000000000000..0f6bef921a4e2 --- /dev/null +++ b/docs/content/docs/connectors/table/pulsar.md @@ -0,0 +1,475 @@ +# Pulsar SQL connector + +Currently, the Pulsar SQL connector is not delivered as a part of the official Flink binary. It is maintained by [StreamNative](https://streamnative.io/). + +The Pulsar SQL connector allows you to query data from or write data into Pulsar topics using simple SQL queries or the [Flink Table API](https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/tableapi/). + +## Dependencies + +This table outlines the Maven dependency that is added to the Pulsar SQL connector. + + + + + + + + + +
Maven Dependency SQL JAR
+ +```xml + + + io.streamnative.connectors + flink-sql-connector-pulsar + 1.15.0.1 + +``` + + + +[SQL JAR](https://repo1.maven.org/maven2/io/streamnative/connectors/flink-sql-connector-pulsar/) + +
+ +## Pulsar SQL connector options + +This table outlines options for the Pulsar SQL connector. + +| Key | Default | Type | Description | +|--------------------------------|----------------------------|--------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `admin-url` | (none) | String | The Pulsar service HTTP URL for the admin endpoint. For example, `http://my-broker.example.com:8080` or `https://my-broker.example.com:8443` for TLS. | +| `explicit` | true | Boolean | Indicate if the table is an explicit Flink table. | +| `key.fields` | | List<String> | An explicit list of physical columns from the table schema that are decoded/encoded from the key bytes of a Pulsar message. By default, this list is empty and thus a key is undefined. | +| `key.format` | (none) | String | The format that is used to deserialize and serialize the key bytes of Pulsar messages. The format identifier is used to discover a suitable format factory. | +| `service-url` | (none) | String | The Service URL for the Pulsar service. To connect to a Pulsar cluster using a client library, you need to specify a Pulsar protocol URL. You can assign a Pulsar protocol URL to a specific cluster.
-This is an example URL of the `localhost:pulsar://localhost:6650`.
- If you have multiple brokers, the URL is something like `pulsar://localhost:6550,localhost:6651,localhost:6652`.
- A URL for a production Pulsar cluster is something like `pulsar://pulsar.us-west.example.com:6650`.
- If TLS authentication is enabled, the URL is something like `pulsar+ssl://pulsar.us-west.example.com:6651`. | +| `sink.custom-topic-router` | (none) | String | (Optional) the custom topic router class URL that is used in the [Pulsar DataStream sink connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-sink). If this option is provided, the `sink.topic-routing-mode` option will be ignored. | +| `sink.message-delay-interval` | 0 ms | Duration | (Optional) the message delay delivery interval that is used in the [Pulsar DataStream sink connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-sink). | +| `sink.topic-routing-mode` | round-robin | Enum | (Optional) the topic routing mode. Available options are `round-robin` and `message-key-hash`. By default, it is set to `round-robin`. If you want to use a custom topic router, use the `sink.custom-topic-router` option to determine the partition for a particular message.
- `round-robin`: the producer publishes messages across all partitions in a round-robin fashion to achieve the maximum throughput. The round-robin method is not implemented for individual messages. However, it is set to the same boundary of the `pulsar.producer.batchingMaxMessages` option to make batching take effect.
- `message-key-hash`: if no key is provided, the partitioned producer will randomly pick one single topic partition and publish all messages to that partition. If a key is provided for a message, the partitioned producer will hash the key and assign the message to a particular partition. | +| `source.start.message-id` | (none) | String | (Optional) the message ID that is used to specify a starting point for the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source) to consume data. Available options are `earliest`, `latest`, and the message ID (in a format of `ledgerId:entryId:partitionId`, such as "12:2:-1"). | +| `source.start.publish-time` | (none) | Long | (Optional) the publish timestamp that is used to specify a starting point for the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source) to consume data. | +| `source.subscription-name` | flink-sql-connector-pulsar | String | The subscription name of the consumer that is used by the runtime [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source). This argument is required for constructing the consumer. | +| `source.subscription-type` | Exclusive | Enum | The [subscription type](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-subscriptions) that is supported by the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source). Currently, only `Exclusive` and `Shared` subscription types are supported. | +| `source.stop.at-message-id` | (none) | String | (Optional) message id used to specify a stop cursor for the unbounded sql source. Use `never"`, `latest` or pass in a message id representation in `ledgerId:entryId:partitionId`, such as `12:2:-1` | +| `source.stop.at-publish-time` | (none) | Long | (Optional) publish timestamp used to specify a stop cursor for the unbounded sql source. | +| `source.stop.after-message-id` | (none) | String | (Optional) message id used to specify a stop position but include the given message in the consuming result for the unbounded sql source. Pass in a message id representation in `ledgerId:entryId:partitionId`, such as `12:2:-1`. | +| `topics` | (none) | List<String> | Topic name(s) the table reads data from. It can be a single topic name or a list of topic names separated by a semicolon symbol (`;`) like `topic-1;topic-2`. | + +However, besides these Pulsar SQL connector options, you can configure the underlying Pulsar DataStream Connector using the connector options defined in [Apache Pulsar Source Connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#source-configurable-options) and [Apache Pulsar Sink Connector]([subscription type]([Apache Pulsar Source Connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#sink-configurable-options) using `WITH` clause. + +## Features + +This section describes some significant features for the Pulsar SQL connector. + +### Available metadata + +The connector metadata are the metadata specific to the external system in use. [Flink SQL](https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/fsql/overview/) supports using metadata from the external system or their decoding/encoding formats as a table column. + +This table outlines the Pulsar topic metadata that can be mapped by the Pulsar SQL connector to Flink table fields. + +> **Note** +> +> - The `R/W` column defines whether a metadata is readable from the Pulsar topics (`R`) and/or writable to the Pulsar topics (`W`). +> - The `R` column defines that a metadata is read-only. The read-only metadata must be declared `VIRTUAL` to exclude them during an `INSERT INTO` operation. + +| Key | Data Type | R/W | +|:--------------|:-----------------------------|-----| +| topic | STRING NOT NULL | R | +| message_size | INT NOT NULL | R | +| producer_name | STRING NOT NULL | R | +| message_id | BYTES NOT NULL | R | +| sequenceId | BIGINT NOT NULL | R | +| publish_time | TIMESTAMP_LTZ(3) NOT NULL | R | +| event_time | TIMESTAMP_LTZ(3) NOT NULL | R/W | +| properties | MAP NOT NULL | R/W | + + +The extended `CREATE TABLE` example demonstrates the syntax for exposing `publish_time`, `producer_name`, and `topic` metadata fields. + +```sql +CREATE TABLE user +( + `publish_time` TIMESTAMP(3) METADATA VIRTUAL FROM 'publish_time', + `producer_name` STRING METADATA VIRTUAL, + `topic` STRING METADATA VIRTUAL, + `uid` BIGINT, + `item_id` BIGINT, + `description` STRING +) WITH ( + 'connector' = 'pulsar', + 'topics' = 'persistent://public/default/user', + 'service-url' = 'pulsar://localhost:6650', + 'admin-url' = 'http://localhost:8080', + 'source.start.message-id' = 'earliest' , + 'format' = 'json' +); +``` + +### Key and value formats + +You can serialize or deserialize the key and the value of a Pulsar message to or from raw bytes using one of the Flink official [formats](https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/connectors/table/formats/overview/). For details about how to configure the key and value formats, see [configure key and value formats](#configure-key-and-value-formats). + +#### Value format + +If the key of a Pulsar message is absent or ignored, you should use the `format` option (a synonym for `value.format`) to designate a decoded/encoded format for the Pulsar message. For details, see the [formats documentation](https://docs.google.com/document/d/1itz4cyTYVmxf3g_iBYyh6rN092d4Se5-66X0kAP6ERo/edit#). All format options are prefixed with the format identifier, such as `json.ignore-parse-errors`. + +```sql +CREATE TABLE users +( + `uid` BIGINT, + `item_id` BIGINT, + `description` STRING +) WITH ( + 'connector' = 'pulsar', + 'topics' = 'persistent://public/default/users', + 'service-url' = 'pulsar://localhost:6650', + ... + 'format' = 'json', + 'json.ignore-parse-errors' = 'true' +); +``` + +You can use the value format to deserialize a Pulsar message to the following Flink data types: + +```text +ROW<`uid` BIGINT, `item_id` BIGINT, `description` STRING> +``` + +#### Key format + +The deserialized or serialized key of a Pulsar message can be mapped to a Flink table field. The key format includes the fields that are listed in the `key.fields` option. You can use the semicolon symbol (`;`) to separate multiple fields. Thus, the key format is configured with the following data type: + +```text +ROW<`uid` BIGINT, `item_id` BIGINT> +``` + +And all the key fields are excluded in the value format's data type: + +```text +ROW<`description` STRING> +``` + +### Topic and partition discovery + +The `topics` and `topic-pattern` options specify the topics and topic pattern for the [Pulsar DataStream connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/) to consume or produce data. + +- `topics`: a list of topics that are separated using the semicolon symbol (`;`) like `topic-1;topic-2`. + +- `topic-pattern`: uses the regular expression to discover the matched topics. For example, if the `topic-pattern` is set to `test-topic-[0-9]`, then all topics whose names match the specified regular expression (starting with `test-topic-` and ending with a single digit) will be subscribed to by the consumer when running a query job. + +By default, the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source) dynamically checks topic changes every 30 seconds, such as the number of topic partitions and topics. To disable automatic topic and partition discovery, you can set a negative value for the `pulsar.source.partitionDiscoveryIntervalMs` option. For details, see [Pulsar DataStream connector documentation](https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/connectors/datastream/pulsar/). + +### Starting position + +You can use the `source.start.message-id` or `source.start-publish-time` option to designate the starting position for the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source). + +You can configure one of the following values for the `source.start.message-id` option: + +- `earliest` +- `latest` +- A message ID (in `::` format) + +You can configure the `source.start.publish-time` option with a Long type timestamp value. + +The `source.start.message-id` and `source.start-publish-time` options are exclusive. If both options are configured, the validation will fail. By default, the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source) uses the `StartCursor.earliest()` option. + +### Sink topic router + +The Pulsar DataStream connector supports designating a topic router. In the Pulsar SQL connector, you can configure the sink topic router using the `sink.topic-routing-mode` or `sink.custom-topic-router` option. + +- `sink.topic-routing-mode`: the built-in topic router implementation. Available values are `round-robin` and `message-key-hash`. +- `sink.custom-topic-router`: the full class name of the custom topic router. Make sure that the custom topic router implementation is set on the classpath. + +The `sink.topic-routing-mode` and `sink.custom-topic-router` options are exclusive. If both options are configured, the validation will fail. + +### Data type mapping + +[Pulsar Schema](https://pulsar.apache.org/docs/schema-get-started/) describes how to deserialize data from and serialize data to a message stored in Pulsar topics. Flink SQL also has its own data types as well as the deserialization and serialization framework. The Pulsar SQL connector provides multiple choices to map a Pulsar schema to Flink data types to transfer data between Flink SQL and Pulsar. + +Flink SQL uses [formats](https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/connectors/table/formats/overview/) to decode and encode data from external systems, such as [flink-csvs](https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/connectors/table/formats/csv/), [flink-json](https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/connectors/table/formats/json/), and [flink-avro](https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/connectors/table/formats/avro/). If you want to read a Pulsar topic with a predefined Pulsar schema (JSON, CSV, or Avro schema), you can use the related Flink format. + +This table outlines the recommended Flink format for each Pulsar Schema. To use the `avro` and `json` format, you should add `flink-avro` and `flink-json` to your dependencies if they are not added yet. (By default, the JAR package of the Pulsar SQL connector is bundled with them). + +| Pulsar schema | Flink format | +|-----------------|-------------------| +| AVRO | avro | +| JSON | json | +| PROTOBUF | Not supported yet | +| PROTOBUF_NATIVE | Not supported yet | +| AUTO_CONSUME | Not supported yet | +| AUTO_PUBLISH | Not supported yet | +| NONE/BYTES | raw | +| BOOLEAN | raw | +| STRING | raw | +| DOUBLE | raw | +| FLOAT | raw | +| INT8 | raw | +| INT16 | raw | +| INT32 | raw | +| INT64 | raw | +| LOCAL_DATE | Not supported yet | +| LOCAL_TIME | Not supported yet | +| LOCAL_DATE_TIME | Not supported yet | + +> **Note** +> +> When Flink writes data to a Pulsar topic with the Avro format, the Pulsar consumer cannot consume data with the Avro schema. + +## PulsarCatalog + +PulsarCatalog supports configuring the Pulsar cluster as the metadata storage for Flink tables. + +### Explicit tables and native tables + +PulsarCatalog defines two different kinds of tables: `explicit` tables and `native` tables. + +- An `explicit` table is a table explicitly created using the `CREATE` statements or table API. It is like the common use pattern in other SQL connectors. You can create a table and then query data from or write data to the table. +- A `native` table is automatically created by PulsarCatalog. PulsarCatalog scans all non-system topics in a Pulsar cluster, and then maps each topic into a Flink table without using the `CREATE` statements. + +#### Explicit tables + +PulsarCatalog uses the Pulsar `SchemaInfo` to store the Flink schema bytes for an `explicit table`. + +For each `explicit` table, PulsarCatalog creates a **placehoder** topic under a preconfigured tenant in the Pulsar cluster. The default value for such a preconfigured tenant is named `__flink_catalog`, but you can use the `catalog-tenant` option to specify a different tenant name. The Flink database is then mapped to a namespace with the same name. Finally, the **placehoder** topic that saves the Flink table schema information is named as `table_`. + +For example, if you create a table `users` under the `testdb` database, then a topic named `table_users` is created under the `__flink_catalog` tenant in the`testdb` namespace. + +The table like `table_users` is called a **placeholder topic**, because these topics do not have any producer or consumer. Therefore, no data would flow in or out of these topics. You can use the schema information of such topics to store the Flink table metadata. + +For the JSON format, the Flink table schema is serialized and then stored under the `schema` key. Other table options from `CREATE TABLE xxx WITH ()` statements are stored under the `properties` key. + +You can use the `pulsar-admin` CLI tool to retrieve the JSON format of a topic schema: + +```bash +pulsar-admin schemas get persistent://// +``` + +#### Native tables + +A `native` table does not have any **placeholder topic**. Instead, PulsarCatalog maps the Pulsar schema of the `native table` to a Flink table schema when creating a SQL job. + +This table outlines the Flink formats that are used for a native table's Pulsar schema. + +| Pulsar schema | Flink data type | Flink format | Work or not | +|-----------------|-------------------------------------------|-------------------|-------------| +| AVRO | It is decided by the Avro format. | avro | Yes | +| JSON | It is decided by the JSON format. | json | Yes | +| PROTOBUF | Not supported yet | / | No | +| PROTOBUF_NATIVE | It is decided by the Protobuf definition. | Not supported yet | No | +| AUTO_CONSUME | Not supported yet | / | No | +| AUTO_PUBLISH | Not supported yet | / | No | +| NONE/BYTES | DataTypes.BYTES() | raw | Yes | +| BOOLEAN | DataTypes.BOOLEAN() | raw | Yes | +| LOCAL_DATE | DataTypes.DATE() | / | No | +| LOCAL_TIME | DataTypes.TIME() | / | No | +| LOCAL_DATE_TIME | DataTypes.TIMESTAMP(3) | / | No | +| STRING | DataTypes.STRING() | raw | Yes | +| DOUBLE | DataTypes.DOUBLE() | raw | Yes | +| FLOAT | DataTypes.FLOAT() | raw | Yes | +| INT8 | DataTypes.TINYINT() | raw | Yes | +| INT16 | DataTypes.SMALLINT() | raw | Yes | +| INT32 | DataTypes.INT() | raw | Yes | +| INT64 | DataTypes.BIGINT() | raw | Yes | + +> **Note** +> +> Even if there are corresponding Flink data formats for the `LOCAL_DATE`, `LOCAL_TIME`, and `LOCAL_DATE_TIME` options, the `raw` format is not able to decode messages with these Pulsar schemas. In this case, the auto schema mapping will fail. + +#### Comparison + +After a `native` table is created, you can query data from existing Pulsar topics. PulsarCatalog automatically reads the topic's schema and decides which decoded/encoded format to use. However, the `native` table does not support watermarks and primary keys. Therefore, you cannot use the `native` table to do window aggregation. A `native` table maps `tenant/namespace` to a database and the topic name to the table name. + +To fully manage a table, you can use the `explicit` table to define watermark fields, specify metadata fields, and specify a custom format. The usage is similar to creating a Pulsar table in `GenericInMemoryCatalog`. You can bind an `explicit` table to a Pulsar topic and each Pulsar topic can be bound to multiple Flink tables (including the `native` table). + +### PulsarCatalog options + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescriptionRequired
catalog-admin-url
"http://localhost:8080"String(Required) The admin URL of the Pulsar cluster.Yes
catalog-auth-params
(none)StringThe authentication parameters for accessing the Pulsar cluster.
catalog-auth-plugin
(none)StringThe name of the authentication plugin for accessing the Pulsar cluster.
catalog-service-url
"pulsar://localhost:6650"String(Required) The service URL of the Pulsar cluster.Yes
catalog-tenant
"__flink_catalog"StringThe Pulsar tenant that stores all table information.
default-database
"default"StringThe default database when using PulsarCatalog. It will be created if it does not exist.
+ +### Create PulsarCatalog + +You can create PulsarCatalog using any of the following ways: + +- SQL query statements +- Flink [Table API](https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/common/) + +This example shows how to create PulsarCatalog using the SQL query statements. + +```sql +CREATE +CATALOG pulsar + WITH ( + 'type' = 'pulsar-catalog', + 'catalog-admin-url' = '', + 'catalog-service-url' = '' +); +``` + +This example shows how to create PulsarCatalog using the Flink [Table API](https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/common/). + +```java +Catalog pulsarCatalog = new PulsarCatalog(); +tableEnv.registerCatalog("pulsar",pulsarCatalog); +``` + +> **Note** +> +> If you use the SQL Client and configure catalogs in `sql-clients-defaults.yaml` YAML file, make sure that this file has been removed due to [FLIP-163](https://cwiki.apache.org/confluence/display/FLINK/FLIP-163%3A+SQL+Client+Improvements) and this file is no longer available for Flink 1.15. + +## Examples + +This section lists some examples about the Pulsar SQL connector. + +### Create a table + +This example shows how to use the Pulsar SQL connector to create a table in a JSON encoding/decoding format. + +```sql +CREATE TABLE users +( + `user_id` BIGINT, + `item_id` BIGINT, + `description` STRING, + `ts` TIMESTAMP(3) METADATA FROM 'timestamp' +) WITH ( + 'connector' = 'pulsar', + 'topics' = 'persistent://public/default/users', + 'service-url' = 'pulsar://localhost:6650', + 'admin-url' = 'http://localhost:8080', + 'format' = 'json' +) +``` + +### Configure key and value formats + +This example shows how to specify and configure the key format and value formats. The format options are prefixed with either the `key` or `value` plus the format identifier. + +```sql +CREATE TABLE users +( + `uid` BIGINT, + `item_id` BIGINT, + `description` STRING +) WITH ( + 'connector' = 'pulsar', + 'topics' = 'persistent://public/default/users', + 'service-url' = 'pulsar://localhost:6650', + 'admin-url' = 'http://localhost:8080', + 'key.format' = 'json', + 'key.json.ignore-parse-errors' = 'true', + 'key.fields' = 'uid;item_id', + 'value.format' = 'json', + 'value.json.fail-on-missing-field' = 'false' +``` + +### Configure authentication parameters + +This example shows how to specify and configure the authentication parameters used by the underlying Pulsar admin client. T + +```sql +CREATE TABLE users +( + `uid` BIGINT, + `item_id` BIGINT, + `description` STRING +) WITH ( + 'connector' = 'pulsar', + 'topics' = 'persistent://public/default/users', + 'service-url' = 'pulsar://localhost:6650', + 'admin-url' = 'http://localhost:8080', + 'pulsar.client.tlsAllowInsecureConnection' = 'true', + 'pulsar.client.authPluginClassName' = 'org.apache.pulsar.client.impl.auth.AuthenticationToken', + 'pulsar.client.authParams' = 'token:eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJ1c2VyMSJ9.2AgtxHe8-2QBV529B5DrRtpuqP6RJjrk21Mhn' +``` + +## Relationship between the `apache/flink` repository and `streamnative/flink` repository + +StreamNative forks the [apache/flink](https://github.com/apache/flink) repository and maintains the project in [streamnative/flink](https://github.com/streamnative/flink). Compared with the official Flink repository, the StreamNative forked repository maintains a different Pulsar DataStream Connector and Pulsar SQL Connector. + +Currently, the `apache/flink` repository only contains the Pulsar DataStream connector while the `streamnative/flink` repository contains both the Pulsar DataStream connector and the Pulsar SQL connector. + +- Pulsar DataStream connector: the code and distribution in the `streamnative/flink` repository differ slightly from those in the `apache/flink` repository. Usually, new features and bug fixes will first go into the `streamnative/flink` repository, and then related PRs are submitted to the `apache/flink` repository. +- Pulsar SQL connector: StreamNative will merge the Pulsar SQL connector back to the Flink community in the coming months. + +For detailed differences and recommendations, see the following sections. + +### Components + +| Repository | Pulsar DataStream source | Pulsar DataStream sink | Pulsar SQL source | Pulsar SQL sink | +|-------------------------|--------------------------|------------------------|-------------------|-----------------| +| apache/flink 1.14 | Yes | | | | +| apache/flink 1.15 | Yes | Yes | | | +| streamnative/flink 1.15 | Yes | Yes | Yes | Yes | + +### Release process, version, and documentation + +| Repository | Release | Versioning | DataStream connector documentation | SQL connector documentation | +|-------------------|----------------------------------------------------------|------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------|-----------------------------| +| apache/flink | Managed by Flink community, following Flink release process | Standard Semantic Versioning, like 1.15.0 | Maintained on [Official Flink Website](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/) | Not available yet | +| streamnative/flink | Managed by StreamNative, following a monthly release process. | Use an extra digit on top of the Flink version, such as 1.15.0.1 | Not available yet, but similar to the [Official Flink Website](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/) | Maintained in current documentation | + +### Issue and Feature tracking + +- For the Pulsar DataStream connector, use [Flink JIRA](https://issues.apache.org/jira/projects/FLINK/issues) to submit bug reports and feature requests. +- For the Pulsar SQL connector, since it's not merged back to the official Flink repository yet, create an [issue](https://github.com/streamnative/flink/issues) in the `streamnative/flink` repository to submit bug reports and feature requests. Once the SQL Connector is merged back to the official Flink repository, use the Flink JIRA as well. + +### Artifact locations + +| Repository | Pulsar DataStream connector | Pulsar SQL connector | +|----------------------|-------------------------------------------------------------------------------------|-----------------------------------------------------------------------------------------| +| apache/flink | https://search.maven.org/artifact/org.apache.flink/flink-connector-pulsar | | +| streamnative/flink | https://search.maven.org/artifact/io.streamnative.connectors/flink-connector-pulsar | https://search.maven.org/artifact/io.streamnative.connectors/flink-sql-connector-pulsar | diff --git a/docs/content/docs/deployment/cli.md b/docs/content/docs/deployment/cli.md index 983a941d16ba9..7a8e0ad04abf1 100644 --- a/docs/content/docs/deployment/cli.md +++ b/docs/content/docs/deployment/cli.md @@ -416,6 +416,22 @@ $ ./bin/flink run \ --python examples/python/table/word_count.py ``` +- Run a PyFlink job using a [YARN cluster in Application Mode]({{< ref "docs/deployment/resource-providers/yarn" >}}#application-mode): +```bash +$ ./bin/flink run-application -t yarn-application \ + -Djobmanager.memory.process.size=1024m \ + -Dtaskmanager.memory.process.size=1024m \ + -Dyarn.application.name= \ + -Dyarn.ship-files=/path/to/shipfiles \ + -pyarch shipfiles/venv.zip \ + -pyclientexec venv.zip/venv/bin/python3 \ + -pyexec venv.zip/venv/bin/python3 \ + -py shipfiles/word_count.py +``` +Note It assumes that the Python dependencies needed to execute the job are already placed in the directory `/path/to/shipfiles`. For example, it should contain venv.zip and word_count.py for the above example. + +Note As it executes the job on the JobManager in YARN application mode, the paths specified in `-pyarch` and `-py` are paths relative to `shipfiles` which is the directory name of the shipped files. + - Run a PyFlink application on a native Kubernetes cluster having the cluster ID ``, it requires a docker image with PyFlink installed, please refer to [Enabling PyFlink in docker]({{< ref "docs/deployment/resource-providers/standalone/docker" >}}#enabling-python): ```bash $ ./bin/flink run-application \ diff --git a/docs/content/docs/deployment/config.md b/docs/content/docs/deployment/config.md index 9a00287658e3e..a38e72d0e1828 100644 --- a/docs/content/docs/deployment/config.md +++ b/docs/content/docs/deployment/config.md @@ -167,7 +167,7 @@ The JobManager ensures consistency during recovery across TaskManagers. For the {{< generated/common_high_availability_section >}} -**Options for the Job Result Store in high-availability setups** +**Options for the JobResultStore in high-availability setups** {{< generated/common_high_availability_jrs_section >}} diff --git a/docs/content/docs/deployment/elastic_scaling.md b/docs/content/docs/deployment/elastic_scaling.md index 47e220d6cb44c..386f083334015 100644 --- a/docs/content/docs/deployment/elastic_scaling.md +++ b/docs/content/docs/deployment/elastic_scaling.md @@ -151,5 +151,48 @@ The behavior of Adaptive Scheduler is configured by [all configuration options c - **Unused slots**: If the max parallelism for slot sharing groups is not equal, slots offered to Adaptive Scheduler might be unused. - Scaling events trigger job and task restarts, which will increase the number of Task attempts. +## Adaptive Batch Scheduler + +The Adaptive Batch Scheduler can automatically decide parallelisms of operators for batch jobs. If an operator is not set with a parallelism, the scheduler will decide parallelism for it according to the size of its consumed datasets. This can bring many benefits: +- Batch job users can be relieved from parallelism tuning +- Automatically tuned parallelisms can better fit consumed datasets which have a varying volume size every day +- Operators from SQL batch jobs can be assigned with different parallelisms which are automatically tuned + +### Usage + +To automatically decide parallelisms for operators with Adaptive Batch Scheduler, you need to: +- Configure to use Adaptive Batch Scheduler. +- Set the parallelism of operators to `-1`. + +#### Configure to use Adaptive Batch Scheduler +To use Adaptive Batch Scheduler, you need to: +- Set `jobmanager.scheduler: AdaptiveBatch`. +- Leave the [`execution.batch-shuffle-mode`]({{< ref "docs/deployment/config" >}}#execution-batch-shuffle-mode) unset or explicitly set it to `ALL-EXCHANGES-BLOCKING` (default value) due to ["ALL-EXCHANGES-BLOCKING jobs only"](#limitations-2). + +In addition, there are several related configuration options that may need adjustment when using Adaptive Batch Scheduler: +- [`jobmanager.adaptive-batch-scheduler.min-parallelism`]({{< ref "docs/deployment/config" >}}#jobmanager-adaptive-batch-scheduler-min-parallelism): The lower bound of allowed parallelism to set adaptively. Currently, this option should be configured as a power of 2, otherwise it will be rounded up to a power of 2 automatically. +- [`jobmanager.adaptive-batch-scheduler.max-parallelism`]({{< ref "docs/deployment/config" >}}#jobmanager-adaptive-batch-scheduler-max-parallelism): The upper bound of allowed parallelism to set adaptively. Currently, this option should be configured as a power of 2, otherwise it will be rounded down to a power of 2 automatically. +- [`jobmanager.adaptive-batch-scheduler.avg-data-volume-per-task`]({{< ref "docs/deployment/config" >}}#jobmanager-adaptive-batch-scheduler-avg-data-volume-per-task): The average size of data volume to expect each task instance to process. Note that since the parallelism of the vertices is adjusted to a power of 2, the actual average size will be 0.75~1.5 times this value. It is also important to note that when data skew occurs, or the decided parallelism reaches the max parallelism (due to too much data), the data actually processed by some tasks may far exceed this value. +- [`jobmanager.adaptive-batch-scheduler.default-source-parallelism`]({{< ref "docs/deployment/config" >}}#jobmanager-adaptive-batch-scheduler-default-source-parallelism): The default parallelism of data source. + +#### Set the parallelism of operators to `-1` +Adaptive Batch Scheduler will only decide parallelism for operators whose parallelism is not specified by users (parallelism is `-1`). So if you want the parallelism of operators to be decided automatically, you should configure as follows: +- Set `parallelism.default: -1` +- Set `table.exec.resource.default-parallelism: -1` in SQL jobs. +- Don't call `setParallelism()` for operators in DataStream/DataSet jobs. +- Don't call `setParallelism()` on `StreamExecutionEnvironment/ExecutionEnvironment` in DataStream/DataSet jobs. + +### Performance tuning + +1. It's recommended to use [Sort Shuffle](https://flink.apache.org/2021/10/26/sort-shuffle-part1.html) and set [`taskmanager.network.memory.buffers-per-channel`]({{< ref "docs/deployment/config" >}}#taskmanager-network-memory-buffers-per-channel) to `0`. This can decouple the required network memory from parallelism, so that for large scale jobs, the "Insufficient number of network buffers" errors are less likely to happen. +2. It's recommended to set [`jobmanager.adaptive-batch-scheduler.max-parallelism`]({{< ref "docs/deployment/config" >}}#jobmanager-adaptive-batch-scheduler-max-parallelism) to the parallelism you expect to need in the worst case. Values larger than that are not recommended, because excessive value may affect the performance. This option can affect the number of subpartitions produced by upstream tasks, large number of subpartitions may degrade the performance of hash shuffle and the performance of network transmission due to small packets. + +### Limitations + +- **Batch jobs only**: Adaptive Batch Scheduler only supports batch jobs. Exception will be thrown if a streaming job is submitted. +- **ALL-EXCHANGES-BLOCKING jobs only**: At the moment, Adaptive Batch Scheduler only supports jobs whose [shuffle mode]({{< ref "docs/deployment/config" >}}#execution-batch-shuffle-mode) is `ALL-EXCHANGES-BLOCKING`. +- **The decided parallelism will be a power of 2**: In order to ensure downstream tasks to consume the same count of subpartitions, the configuration option [`jobmanager.adaptive-batch-scheduler.max-parallelism`]({{< ref "docs/deployment/config" >}}#jobmanager-adaptive-batch-scheduler-max-parallelism) should be set to be a power of 2 (2^N), and the decided parallelism will also be a power of 2 (2^M and M <= N). +- **FileInputFormat sources are not supported**: FileInputFormat sources are not supported, including `StreamExecutionEnvironment#readFile(...)` `StreamExecutionEnvironment#readTextFile(...)` and `StreamExecutionEnvironment#createInput(FileInputFormat, ...)`. Users should use the new sources([FileSystem DataStream Connector]({{< ref "docs/connectors/datastream/filesystem.md" >}}) or [FileSystem SQL Connector]({{< ref "docs/connectors/table/filesystem.md" >}})) to read files when using the Adaptive Batch Scheduler. +- **Inconsistent broadcast results metrics on WebUI**: In Adaptive Batch Scheduler, for broadcast results, the number of bytes/records sent by the upstream task counted by metric is not equal to the number of bytes/records received by the downstream task, which may confuse users when displayed on the Web UI. See [FLIP-187](https://cwiki.apache.org/confluence/display/FLINK/FLIP-187%3A+Adaptive+Batch+Job+Scheduler) for details. {{< top >}} diff --git a/docs/content/docs/deployment/filesystems/azure.md b/docs/content/docs/deployment/filesystems/azure.md index 1b94c6d3b2ba4..c9d49f52a4177 100644 --- a/docs/content/docs/deployment/filesystems/azure.md +++ b/docs/content/docs/deployment/filesystems/azure.md @@ -64,7 +64,7 @@ See below for how to use Azure Blob Storage in a Flink job: env.readTextFile("wasb://@$.blob.core.windows.net/"); // Write to Azure Blob storage -stream.writeAsText("wasb://@$.blob.core.windows.net/") +stream.writeAsText("wasb://@$.blob.core.windows.net/"); // Use Azure Blob Storage as checkpoint storage env.getCheckpointConfig().setCheckpointStorage("wasb://@$.blob.core.windows.net/"); diff --git a/docs/content/docs/deployment/filesystems/oss.md b/docs/content/docs/deployment/filesystems/oss.md index 7ca5129446fb2..0dc0ebabc09f1 100644 --- a/docs/content/docs/deployment/filesystems/oss.md +++ b/docs/content/docs/deployment/filesystems/oss.md @@ -47,7 +47,7 @@ Below shows how to use OSS in a Flink job: env.readTextFile("oss:///"); // Write to OSS bucket -stream.writeAsText("oss:///") +stream.writeAsText("oss:///"); // Use OSS as checkpoint storage env.getCheckpointConfig().setCheckpointStorage("oss:///"); diff --git a/docs/content/docs/deployment/ha/overview.md b/docs/content/docs/deployment/ha/overview.md index 36cdcb460820c..1939474680a26 100644 --- a/docs/content/docs/deployment/ha/overview.md +++ b/docs/content/docs/deployment/ha/overview.md @@ -82,14 +82,16 @@ Once this happens, all the HA data, including the metadata stored in the HA serv ## JobResultStore -In order to preserve a job's scheduling status across failover events and prevent erroneous -re-execution of globally terminated (i.e. finished, cancelled or failed) jobs, Flink persists -status of terminated jobs to a filesystem using the JobResultStore. -The JobResultStore allows job results to outlive a finished job, and can be used by -Flink components involved in the recovery of a highly-available cluster in order to -determine whether a job should be subject to recovery. - -The JobResultStore has sensible defaults for its behaviour, such as result storage -location, but these can be [configured]({{< ref "docs/deployment/config#high-availability" >}}). +The JobResultStore is used to archive the final result of a job that reached a globally-terminal +state (i.e. finished, cancelled or failed). The data is stored on a file system (see +[job-result-store.storage-path]({{< ref "docs/deployment/config#job-result-store-storage-path" >}})). +Entries in this store are marked as dirty as long as the corresponding job wasn't cleaned up properly +(artifacts are found in the job's subfolder in [high-availability.storageDir]({{< ref "docs/deployment/config#high-availability-storagedir" >}})). + +Dirty entries are subject to cleanup, i.e. the corresponding job is either cleaned up by Flink at +the moment or will be picked up for cleanup as part of a recovery. The entries will be deleted as +soon as the cleanup succeeds. Check the JobResultStore configuration parameters under +[HA configuration options]({{< ref "docs/deployment/config#high-availability" >}}) for further +details on how to adapt the behavior. {{< top >}} diff --git a/docs/content/docs/deployment/memory/network_mem_tuning.md b/docs/content/docs/deployment/memory/network_mem_tuning.md index 4bedad5bfb9c1..9c537517b8905 100644 --- a/docs/content/docs/deployment/memory/network_mem_tuning.md +++ b/docs/content/docs/deployment/memory/network_mem_tuning.md @@ -128,7 +128,11 @@ The default settings for exclusive buffers and floating buffers should be suffic The buffer collects records in order to optimize network overhead when sending the data portion to the next subtask. The next subtask should receive all parts of the record before consuming it. -If the buffer size is too small (i.e. less than one record), this can lead to low throughput since the overhead is still pretty large. +If the buffer size is too small, or the buffers are flushed too frequently (`execution.buffer-timeout` configuration parameter), this can lead to decreased throughput +since the per-buffer overhead are significantly higher then per-record overheads in the Flink's runtime. + +As a rule of thumb, we don't recommend thinking about increasing the buffer size, or the buffer timeout unless you can observe a network bottleneck in your real life workload +(downstream operator idling, upstream backpressured, output buffer queue is full, downstream input queue is empty). If the buffer size is too large, this can lead to: - high memory usage diff --git a/docs/content/docs/deployment/overview.md b/docs/content/docs/deployment/overview.md index 64bac268ad8d1..247605ee811ac 100644 --- a/docs/content/docs/deployment/overview.md +++ b/docs/content/docs/deployment/overview.md @@ -158,7 +158,12 @@ When deploying Flink, there are often multiple options available for each buildi Once a job has reached a globally terminal state of either finished, failed or cancelled, the external component resources associated with the job are then cleaned up. In the event of a failure when cleaning up a resource, Flink will attempt to retry the cleanup. You can -[configure]({{< ref "docs/deployment/config#retryable-cleanup" >}}) the retry strategy used. +[configure]({{< ref "docs/deployment/config#retryable-cleanup" >}}) the retry strategy used. +Reaching the maximum number of retries without succeeding will leave the job in a dirty state. +Its artifacts would need to be cleaned up manually (see the +[High Availability Services / JobResultStore]({{< ref "docs/deployment/ha/overview#jobresultstore" >}}) +section for further details). Restarting the very same job (i.e. using the same +job ID) will result in the cleanup being restarted without running the job again. There is currently an issue with the cleanup of CompletedCheckpoints that failed to be deleted while subsuming them as part of the usual CompletedCheckpoint management. These artifacts are diff --git a/docs/content/docs/dev/configuration/overview.md b/docs/content/docs/dev/configuration/overview.md index 4ba7f83c36edd..0ac9dbadf3c03 100644 --- a/docs/content/docs/dev/configuration/overview.md +++ b/docs/content/docs/dev/configuration/overview.md @@ -76,7 +76,10 @@ $ curl https://flink.apache.org/q/quickstart.sh | bash -s {{< version >}} {{< /tab >}} {{< tab "Gradle" >}} -You can create a project with a Gradle build script or use the provided quickstart bash script. +You can create an empty project, where you are required to create the `src/main/java` and +`src/main/resources` directories manually and start writing some class(es) in that, with the use +of the following Gradle build script or instead use the provided quickstart bash script to get a +completely functional startup project. ### Gradle build script @@ -94,25 +97,31 @@ plugins { // artifact properties group = 'org.quickstart' version = '0.1-SNAPSHOT' -mainClassName = 'org.quickstart.StreamingJob' -mainClassName = 'org.quickstart.StreamingJob' +mainClassName = 'org.quickstart.DataStreamJob' description = """Flink Quickstart Job""" ext { javaVersion = '1.8' flinkVersion = '{{< version >}}' + scalaBinaryVersion = '{{< scala_version >}}' slf4jVersion = '1.7.32' log4jVersion = '2.17.1' } sourceCompatibility = javaVersion targetCompatibility = javaVersion tasks.withType(JavaCompile) { - options.encoding = 'UTF-8' + options.encoding = 'UTF-8' } applicationDefaultJvmArgs = ["-Dlog4j.configurationFile=log4j2.properties"] // declare where to find the dependencies of your project repositories { mavenCentral() + maven { + url "https://repository.apache.org/content/repositories/snapshots" + mavenContent { + snapshotsOnly() + } + } } // NOTE: We cannot use "compileOnly" or "shadow" configurations since then we could not run code // in the IDE or with "gradle run". We also cannot exclude transitive dependencies from the @@ -139,10 +148,9 @@ dependencies { // connectors. These must be in the flinkShadowJar configuration! // -------------------------------------------------------------- //flinkShadowJar "org.apache.flink:flink-connector-kafka:${flinkVersion}" + runtimeOnly "org.apache.logging.log4j:log4j-slf4j-impl:${log4jVersion}" runtimeOnly "org.apache.logging.log4j:log4j-api:${log4jVersion}" runtimeOnly "org.apache.logging.log4j:log4j-core:${log4jVersion}" - runtimeOnly "org.apache.logging.log4j:log4j-slf4j-impl:${log4jVersion}" - runtimeOnly "org.slf4j:slf4j-log4j12:${slf4jVersion}" // Add test dependencies here. // testCompile "junit:junit:4.12" } @@ -156,6 +164,13 @@ sourceSets { } run.classpath = sourceSets.main.runtimeClasspath +jar { + manifest { + attributes 'Built-By': System.getProperty('user.name'), + 'Build-Jdk': System.getProperty('java.version') + } +} + shadowJar { configurations = [project.configurations.flinkShadowJar] } diff --git a/docs/content/docs/dev/dataset/examples.md b/docs/content/docs/dev/dataset/examples.md index cd3b42d09b121..6aaf729e9201e 100644 --- a/docs/content/docs/dev/dataset/examples.md +++ b/docs/content/docs/dev/dataset/examples.md @@ -134,7 +134,7 @@ ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); // read the pages and initial ranks by parsing a CSV file DataSet> pagesWithRanks = env.readCsvFile(pagesInputPath) - .types(Long.class, Double.class) + .types(Long.class, Double.class); // the links are encoded as an adjacency list: (page-id, Array(neighbor-ids)) DataSet> pageLinkLists = getLinksDataSet(env); diff --git a/docs/content/docs/dev/dataset/hadoop_map_reduce.md b/docs/content/docs/dev/dataset/hadoop_map_reduce.md index f72da27798b29..04e9b38ff82d4 100644 --- a/docs/content/docs/dev/dataset/hadoop_map_reduce.md +++ b/docs/content/docs/dev/dataset/hadoop_map_reduce.md @@ -85,7 +85,7 @@ The following example shows how to use Hadoop `Mapper` and `Reducer` functions. ```java // Obtain data to process somehow. -DataSet> text = [...] +DataSet> text = [...]; DataSet> result = text // use Hadoop Mapper (Tokenizer) as MapFunction diff --git a/docs/content/docs/dev/dataset/iterations.md b/docs/content/docs/dev/dataset/iterations.md index 5ff217e635879..6366e4fdbeec3 100644 --- a/docs/content/docs/dev/dataset/iterations.md +++ b/docs/content/docs/dev/dataset/iterations.md @@ -171,7 +171,7 @@ IterationState solution = getInitialSolution(); while (!terminationCriterion()) { (delta, workset) = step(workset, solution); - solution.update(delta) + solution.update(delta); } setFinalState(solution); diff --git a/docs/content/docs/dev/dataset/overview.md b/docs/content/docs/dev/dataset/overview.md index bbd33b6b741ed..c86c4f0215a16 100644 --- a/docs/content/docs/dev/dataset/overview.md +++ b/docs/content/docs/dev/dataset/overview.md @@ -281,7 +281,7 @@ It removes the duplicate entries from the input DataSet, with respect to all fie {{< tabs "distinct" >}} {{< tab "Java" >}} ```java -data.distinct() +data.distinct(); ``` {{< /tab >}} {{< tab "Scala" >}} @@ -431,7 +431,7 @@ Produces the union of two data sets. {{< tabs "union" >}} {{< tab "Java" >}} ```java -data.union(data2) +data.union(data2); ``` {{< /tab >}} {{< tab "Scala" >}} @@ -450,7 +450,7 @@ Only Map-like transformations may follow a rebalance transformation. {{< tab "Java" >}} ```java DataSet data1 = // [...] -DataSet> result = data1.rebalance().map(...) +DataSet> result = data1.rebalance().map(...); ``` {{< /tab >}} {{< tab "Scala" >}} @@ -651,7 +651,7 @@ The simplest case is grouping Tuples on one or more fields of the Tuple: {{< tab "Java" >}} ```java DataSet> input = // [...] -UnsortedGrouping,Tuple> keyed = input.groupBy(0) +UnsortedGrouping,Tuple> keyed = input.groupBy(0); ``` {{< /tab >}} {{< tab "Scala" >}} @@ -668,7 +668,7 @@ Tuples are grouped on the first field (the one of Integer type). {{< tab "Java" >}} ```java DataSet> input = // [...] -UnsortedGrouping,Tuple> keyed = input.groupBy(0,1) +UnsortedGrouping,Tuple> keyed = input.groupBy(0,1); ``` {{< /tab >}} {{< tab "Scala" >}} @@ -707,7 +707,7 @@ public class WC { public int count; } DataSet words = // [...] -DataSet wordCounts = words.groupBy("word") +DataSet wordCounts = words.groupBy("word"); ``` {{< /tab >}} {{< tab "Scala" >}} @@ -1393,11 +1393,11 @@ final ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); DataSet myInts = env.fromElements(1, 2, 3, 4, 5); // Create a DataSet from any Java collection -List> data = ... +List> data = ...; DataSet> myTuples = env.fromCollection(data); // Create a DataSet from an Iterator -Iterator longIt = ... +Iterator longIt = ...; DataSet myLongs = env.fromCollection(longIt, Long.class); ``` {{< /tab >}} @@ -1495,14 +1495,14 @@ The distributed cache is used as follows: ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); // register a file from HDFS -env.registerCachedFile("hdfs:///path/to/your/file", "hdfsFile") +env.registerCachedFile("hdfs:///path/to/your/file", "hdfsFile"); // register a local executable file (script, executable, ...) -env.registerCachedFile("file:///path/to/exec/file", "localExecFile", true) +env.registerCachedFile("file:///path/to/exec/file", "localExecFile", true); // define your program and execute ... -DataSet input = ... +DataSet input = ...; DataSet result = input.map(new MyMapper()); ... env.execute(); @@ -1709,4 +1709,4 @@ public static final class Tokenizer extends RichFlatMapFunction> out = in.project(2,0); Note that the Java compiler cannot infer the return type of `project` operator. This can cause a problem if you call another operator on a result of `project` operator such as: ```java -DataSet> ds = .... +DataSet> ds = ....; DataSet> ds2 = ds.project(0).distinct(0); ``` diff --git a/docs/content/docs/dev/datastream/application_parameters.md b/docs/content/docs/dev/datastream/application_parameters.md index 2069a3447e68f..3cdf2f3e08187 100644 --- a/docs/content/docs/dev/datastream/application_parameters.md +++ b/docs/content/docs/dev/datastream/application_parameters.md @@ -89,7 +89,7 @@ ParameterTool parameters = // ... parameter.getRequired("input"); parameter.get("output", "myDefaultValue"); parameter.getLong("expectedCount", -1L); -parameter.getNumberOfParameters() +parameter.getNumberOfParameters(); // .. there are more methods available. ``` diff --git a/docs/content/docs/dev/datastream/execution/parallel.md b/docs/content/docs/dev/datastream/execution/parallel.md index caa53ea2f32b0..2fbe5c0377ab0 100644 --- a/docs/content/docs/dev/datastream/execution/parallel.md +++ b/docs/content/docs/dev/datastream/execution/parallel.md @@ -52,7 +52,7 @@ The parallelism of an individual operator, data source, or data sink can be defi ```java final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); -DataStream text = [...] +DataStream text = [...]; DataStream> wordCounts = text .flatMap(new LineSplitter()) .keyBy(value -> value.f0) @@ -99,8 +99,8 @@ of `3`, set the default parallelism of the execution environment as follows: final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(3); -DataStream text = [...] -DataStream> wordCounts = [...] +DataStream text = [...]; +DataStream> wordCounts = [...]; wordCounts.print(); env.execute("Word Count Example"); diff --git a/docs/content/docs/dev/datastream/execution_mode.md b/docs/content/docs/dev/datastream/execution_mode.md index 7837558876766..415a82d7f6af2 100644 --- a/docs/content/docs/dev/datastream/execution_mode.md +++ b/docs/content/docs/dev/datastream/execution_mode.md @@ -343,15 +343,14 @@ others are not supported. Behavior Change in BATCH mode: * "Rolling" operations such as [reduce()]({{< ref "docs/dev/datastream/operators/overview" >}}#reduce) - or [sum()]({{< ref "docs/dev/datastream/operators/overview" >}}#aggregations) - emit an incremental update for every new record that arrives in `STREAMING` + or sum() emit an incremental update for every new record that arrives in `STREAMING` mode. In `BATCH` mode, these operations are not "rolling". They emit only the final result. Unsupported in BATCH mode: -* [Checkpointing]({{< ref "docs/concepts/stateful-stream-processing" >}}#stateful-stream-processing) +* [Checkpointing]({{< ref "docs/concepts/stateful-stream-processing" >}}#checkpointing) and any operations that depend on checkpointing do not work. * [Iterations]({{< ref "docs/dev/datastream/operators/overview" >}}#iterate) diff --git a/docs/content/docs/dev/datastream/experimental.md b/docs/content/docs/dev/datastream/experimental.md index 654b768245db8..75c1c4796d133 100644 --- a/docs/content/docs/dev/datastream/experimental.md +++ b/docs/content/docs/dev/datastream/experimental.md @@ -62,7 +62,7 @@ Code example: {{< tab "Java" >}} ```java StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); -DataStreamSource source = ... +DataStreamSource source = ...; DataStreamUtils.reinterpretAsKeyedStream(source, (in) -> in, TypeInformation.of(Integer.class)) .window(TumblingEventTimeWindows.of(Time.seconds(1))) .reduce((a, b) -> a + b) diff --git a/docs/content/docs/dev/datastream/fault-tolerance/checkpointing.md b/docs/content/docs/dev/datastream/fault-tolerance/checkpointing.md index 3ead62969c3d3..dc039a088d517 100644 --- a/docs/content/docs/dev/datastream/fault-tolerance/checkpointing.md +++ b/docs/content/docs/dev/datastream/fault-tolerance/checkpointing.md @@ -119,7 +119,7 @@ env.getCheckpointConfig().setExternalizedCheckpointCleanup( env.getCheckpointConfig().enableUnalignedCheckpoints(); // sets the checkpoint storage where checkpoint snapshots will be written -env.getCheckpointConfig().setCheckpointStorage("hdfs:///my/checkpoint/dir") +env.getCheckpointConfig().setCheckpointStorage("hdfs:///my/checkpoint/dir"); // enable checkpointing with finished tasks Configuration config = new Configuration(); diff --git a/docs/content/docs/dev/datastream/fault-tolerance/queryable_state.md b/docs/content/docs/dev/datastream/fault-tolerance/queryable_state.md index 6f3b30f4a3268..2a9edc3ffa620 100644 --- a/docs/content/docs/dev/datastream/fault-tolerance/queryable_state.md +++ b/docs/content/docs/dev/datastream/fault-tolerance/queryable_state.md @@ -125,7 +125,7 @@ In a program like the following, all records of the keyed stream will be used to `ValueState.update(value)`: ```java -stream.keyBy(value -> value.f0).asQueryableState("query-name") +stream.keyBy(value -> value.f0).asQueryableState("query-name"); ``` This acts like the Scala API's `flatMapWithState`. diff --git a/docs/content/docs/dev/datastream/fault-tolerance/serialization/types_serialization.md b/docs/content/docs/dev/datastream/fault-tolerance/serialization/types_serialization.md index 61115370fdf19..47055860efb55 100644 --- a/docs/content/docs/dev/datastream/fault-tolerance/serialization/types_serialization.md +++ b/docs/content/docs/dev/datastream/fault-tolerance/serialization/types_serialization.md @@ -500,7 +500,7 @@ env.getConfig().enableForceKryo(); If Kryo is not able to serialize your POJO, you can add a custom serializer to Kryo, using ```java -env.getConfig().addDefaultKryoSerializer(Class type, Class> serializerClass) +env.getConfig().addDefaultKryoSerializer(Class type, Class> serializerClass); ``` There are different variants of these methods available. diff --git a/docs/content/docs/dev/datastream/fault-tolerance/state.md b/docs/content/docs/dev/datastream/fault-tolerance/state.md index 9cc3e23e31b38..5af6e2635af2c 100644 --- a/docs/content/docs/dev/datastream/fault-tolerance/state.md +++ b/docs/content/docs/dev/datastream/fault-tolerance/state.md @@ -432,6 +432,8 @@ will lead to compatibility failure and `StateMigrationException`. - The map state with TTL currently supports null user values only if the user value serializer can handle null values. If the serializer does not support null values, it can be wrapped with `NullableSerializer` at the cost of an extra byte in the serialized form. +- With TTL enabled configuration, the `defaultValue` in `StateDescriptor`, which is atucally already deprecated, will no longer take an effect. This aims to make the semantics more clear and let user manually manage the default value if the contents of the state is null or expired. + #### Cleanup of Expired State By default, expired values are explicitly removed on read, such as `ValueState#value`, and periodically garbage collected diff --git a/docs/content/docs/dev/datastream/operators/joining.md b/docs/content/docs/dev/datastream/operators/joining.md index 50387afad42b4..6e7e46950b297 100644 --- a/docs/content/docs/dev/datastream/operators/joining.md +++ b/docs/content/docs/dev/datastream/operators/joining.md @@ -39,7 +39,7 @@ stream.join(otherStream) .where() .equalTo() .window() - .apply() + .apply(); ``` Some notes on semantics: @@ -65,8 +65,8 @@ import org.apache.flink.streaming.api.windowing.time.Time; ... -DataStream orangeStream = ... -DataStream greenStream = ... +DataStream orangeStream = ...; +DataStream greenStream = ...; orangeStream.join(greenStream) .where() @@ -119,8 +119,8 @@ import org.apache.flink.streaming.api.windowing.time.Time; ... -DataStream orangeStream = ... -DataStream greenStream = ... +DataStream orangeStream = ...; +DataStream greenStream = ...; orangeStream.join(greenStream) .where() @@ -172,8 +172,8 @@ import org.apache.flink.streaming.api.windowing.time.Time; ... -DataStream orangeStream = ... -DataStream greenStream = ... +DataStream orangeStream = ...; +DataStream greenStream = ...; orangeStream.join(greenStream) .where() @@ -244,8 +244,8 @@ import org.apache.flink.streaming.api.windowing.time.Time; ... -DataStream orangeStream = ... -DataStream greenStream = ... +DataStream orangeStream = ...; +DataStream greenStream = ...; orangeStream .keyBy() diff --git a/docs/content/docs/dev/datastream/operators/overview.md b/docs/content/docs/dev/datastream/operators/overview.md index c6503ec6b4dd6..81e183aa1b754 100644 --- a/docs/content/docs/dev/datastream/operators/overview.md +++ b/docs/content/docs/dev/datastream/operators/overview.md @@ -773,7 +773,7 @@ The description can contain detail information about operators to facilitate deb {{< tabs namedescription >}} {{< tab "Java" >}} ```java -someStream.filter(...).setName("filter").setDescription("x in (1, 2, 3, 4) and y > 1") +someStream.filter(...).setName("filter").setDescription("x in (1, 2, 3, 4) and y > 1"); ``` {{< /tab >}} {{< tab "Scala" >}} diff --git a/docs/content/docs/dev/datastream/operators/process_function.md b/docs/content/docs/dev/datastream/operators/process_function.md index 41d10129b542d..52fd53a402991 100644 --- a/docs/content/docs/dev/datastream/operators/process_function.md +++ b/docs/content/docs/dev/datastream/operators/process_function.md @@ -53,7 +53,7 @@ to apply the `ProcessFunction` on a keyed stream: {{< /hint >}} ```java -stream.keyBy(...).process(new MyProcessFunction()) +stream.keyBy(...).process(new MyProcessFunction()); ``` ## Low-level Joins @@ -461,7 +461,7 @@ Stopping a processing-time timer: {{< tabs "5d0d1344-6f51-44f8-b500-ebe863cedba4" >}} {{< tab "Java" >}} ```java -long timestampOfTimerToStop = ... +long timestampOfTimerToStop = ...; ctx.timerService().deleteProcessingTimeTimer(timestampOfTimerToStop); ``` {{< /tab >}} @@ -484,7 +484,7 @@ Stopping an event-time timer: {{< tabs "581e5996-503c-452e-8b2a-a4daeaf4ac88" >}} {{< tab "Java" >}} ```java -long timestampOfTimerToStop = ... +long timestampOfTimerToStop = ...; ctx.timerService().deleteEventTimeTimer(timestampOfTimerToStop); ``` {{< /tab >}} diff --git a/docs/content/docs/dev/datastream/overview.md b/docs/content/docs/dev/datastream/overview.md index 329d0089b421c..3b4d1392d9860 100644 --- a/docs/content/docs/dev/datastream/overview.md +++ b/docs/content/docs/dev/datastream/overview.md @@ -85,11 +85,11 @@ The `StreamExecutionEnvironment` is the basis for all Flink programs. You can obtain one using these static methods on `StreamExecutionEnvironment`: ```java -getExecutionEnvironment() +getExecutionEnvironment(); -createLocalEnvironment() +createLocalEnvironment(); -createRemoteEnvironment(String host, int port, String... jarFiles) +createRemoteEnvironment(String host, int port, String... jarFiles); ``` Typically, you only need to use `getExecutionEnvironment()`, since this will do @@ -136,9 +136,9 @@ an outside system by creating a sink. These are just some example methods for creating a sink: ```java -writeAsText(String path) +writeAsText(String path); -print() +print(); ``` {{< /tab >}} @@ -744,7 +744,7 @@ List> data = ... DataStream> myTuples = env.fromCollection(data); // Create a DataStream from an Iterator -Iterator longIt = ... +Iterator longIt = ...; DataStream myLongs = env.fromCollection(longIt, Long.class); ``` {{< /tab >}} @@ -777,10 +777,10 @@ Flink also provides a sink to collect DataStream results for testing and debuggi {{< tabs "125e228e-13b5-4c77-93a7-c0f436fcdd2f" >}} {{< tab "Java" >}} ```java -import org.apache.flink.streaming.experimental.DataStreamUtils +import org.apache.flink.streaming.experimental.DataStreamUtils; -DataStream> myResult = ... -Iterator> myOutput = DataStreamUtils.collect(myResult) +DataStream> myResult = ...; +Iterator> myOutput = DataStreamUtils.collect(myResult); ``` {{< /tab >}} diff --git a/docs/content/docs/dev/datastream/sources.md b/docs/content/docs/dev/datastream/sources.md index cae739a45acd8..2a9265ab453a3 100644 --- a/docs/content/docs/dev/datastream/sources.md +++ b/docs/content/docs/dev/datastream/sources.md @@ -341,7 +341,7 @@ The `WatermarkStrategy` is passed to the Source during creation in the DataStrea environment.fromSource( Source source, WatermarkStrategy timestampsAndWatermarks, - String sourceName) + String sourceName); ``` The `TimestampAssigner` and `WatermarkGenerator` run transparently as part of the `ReaderOutput`(or `SourceOutput`) so source implementors do not have to implement any timestamp extraction and watermark generation code. diff --git a/docs/content/docs/dev/datastream/user_defined_functions.md b/docs/content/docs/dev/datastream/user_defined_functions.md index 4d6db24604daf..55aa7acaa8c2e 100644 --- a/docs/content/docs/dev/datastream/user_defined_functions.md +++ b/docs/content/docs/dev/datastream/user_defined_functions.md @@ -212,7 +212,7 @@ returned from the `execute()` method of the execution environment completion of the job). ```java -myJobExecutionResult.getAccumulatorResult("num-lines") +myJobExecutionResult.getAccumulatorResult("num-lines"); ``` All accumulators share a single namespace per job. Thus you can use the same accumulator in diff --git a/docs/content/docs/dev/python/datastream/intro_to_datastream_api.md b/docs/content/docs/dev/python/datastream/intro_to_datastream_api.md index ffa71f6d17299..22bec5cd6b0b2 100644 --- a/docs/content/docs/dev/python/datastream/intro_to_datastream_api.md +++ b/docs/content/docs/dev/python/datastream/intro_to_datastream_api.md @@ -78,7 +78,7 @@ def state_access_demo(): # 3. define the execution logic ds = ds.map(lambda a: Row(a % 4, 1), output_type=Types.ROW([Types.LONG(), Types.LONG()])) \ .key_by(lambda a: a[0]) \ - .map(MyMapFunction(), output_type=Types.ROW([Types.LONG(), Types.LONG()])) + .map(MyMapFunction(), output_type=Types.TUPLE([Types.LONG(), Types.LONG()])) # 4. create sink and emit result to sink output_path = '/opt/output/' diff --git a/docs/content/docs/dev/python/debugging.md b/docs/content/docs/dev/python/debugging.md index cd5836578a9b4..a1723e2a55e17 100644 --- a/docs/content/docs/dev/python/debugging.md +++ b/docs/content/docs/dev/python/debugging.md @@ -119,7 +119,7 @@ You can make use of the [`pydevd_pycharm`](https://pypi.org/project/pydevd-pycha You can enable the profile to analyze performance bottlenecks. ```python -t_env.get_config().get_configuration().set_boolean("python.profile.enabled", True) +t_env.get_config().set("python.profile.enabled", "true") ``` Then you can see the profile result in [logs](#accessing-logs) diff --git a/docs/content/docs/dev/python/dependency_management.md b/docs/content/docs/dev/python/dependency_management.md index e27ec80b00820..c12f78130c17f 100644 --- a/docs/content/docs/dev/python/dependency_management.md +++ b/docs/content/docs/dev/python/dependency_management.md @@ -51,12 +51,15 @@ If third-party JARs are used, you can specify the JARs in the Python Table API a # Specify a list of jar URLs via "pipeline.jars". The jars are separated by ";" # and will be uploaded to the cluster. # NOTE: Only local file URLs (start with "file://") are supported. -table_env.get_config().get_configuration().set_string("pipeline.jars", "file:///my/jar/path/connector.jar;file:///my/jar/path/udf.jar") +table_env.get_config().set("pipeline.jars", "file:///my/jar/path/connector.jar;file:///my/jar/path/udf.jar") + +# It looks like the following on Windows: +table_env.get_config().set("pipeline.jars", "file:///E:/my/jar/path/connector.jar;file:///E:/my/jar/path/udf.jar") # Specify a list of URLs via "pipeline.classpaths". The URLs are separated by ";" # and will be added to the classpath during job execution. # NOTE: The paths must specify a protocol (e.g. file://) and users should ensure that the URLs are accessible on both the client and the cluster. -table_env.get_config().get_configuration().set_string("pipeline.classpaths", "file:///my/jar/path/connector.jar;file:///my/jar/path/udf.jar") +table_env.get_config().set("pipeline.classpaths", "file:///my/jar/path/connector.jar;file:///my/jar/path/udf.jar") ``` or in the Python DataStream API as following: @@ -66,6 +69,9 @@ or in the Python DataStream API as following: # NOTE: Only local file URLs (start with "file://") are supported. stream_execution_environment.add_jars("file:///my/jar/path/connector1.jar", "file:///my/jar/path/connector2.jar") +# It looks like the following on Windows: +stream_execution_environment.add_jars("file:///E:/my/jar/path/connector1.jar", "file:///E:/my/jar/path/connector2.jar") + # Use the add_classpaths() to add the dependent jars URLs into the classpath. # The URLs will also be added to the classpath of both the client and the cluster. # NOTE: The paths must specify a protocol (e.g. file://) and users should ensure that the diff --git a/docs/content/docs/dev/python/faq.md b/docs/content/docs/dev/python/faq.md index 592e2a4ed990d..0c73a27f1eecb 100644 --- a/docs/content/docs/dev/python/faq.md +++ b/docs/content/docs/dev/python/faq.md @@ -74,10 +74,10 @@ You can specify the dependencies with the following Python Table APIs or through ```python # NOTE: Only local file URLs (start with "file:") are supported. -table_env.get_config().get_configuration().set_string("pipeline.jars", "file:///my/jar/path/connector.jar;file:///my/jar/path/udf.jar") +table_env.get_config().set("pipeline.jars", "file:///my/jar/path/connector.jar;file:///my/jar/path/udf.jar") # NOTE: The Paths must specify a protocol (e.g. "file") and users should ensure that the URLs are accessible on both the client and the cluster. -table_env.get_config().get_configuration().set_string("pipeline.classpaths", "file:///my/jar/path/connector.jar;file:///my/jar/path/udf.jar") +table_env.get_config().set("pipeline.classpaths", "file:///my/jar/path/connector.jar;file:///my/jar/path/udf.jar") ``` For details about the APIs of adding Java dependency, you can refer to [the relevant documentation]({{< ref "docs/dev/python/dependency_management" >}}#java-dependency-in-python-program) diff --git a/docs/content/docs/dev/python/python_config.md b/docs/content/docs/dev/python/python_config.md index 65d56d874df6b..659ddb1afb02c 100644 --- a/docs/content/docs/dev/python/python_config.md +++ b/docs/content/docs/dev/python/python_config.md @@ -49,9 +49,23 @@ from pyflink.table import TableEnvironment, EnvironmentSettings env_settings = EnvironmentSettings.in_streaming_mode() t_env = TableEnvironment.create(env_settings) +t_env.get_config().set("python.fn-execution.bundle.size", "1000") +``` -config = t_env.get_config().get_configuration() -config.set_integer("python.fn-execution.bundle.size", 1000) +The config options could also be set when creating EnvironmentSettings: +```python +from pyflink.common import Configuration +from pyflink.table import TableEnvironment, EnvironmentSettings + +# create a streaming TableEnvironment +config = Configuration() +config.set_string("python.fn-execution.bundle.size", "1000") +env_settings = EnvironmentSettings \ + .new_instance() \ + .in_streaming_mode() \ + .with_configuration(config) \ + .build() +table_env = TableEnvironment.create(env_settings) ``` ## Python Options diff --git a/docs/content/docs/dev/python/python_execution_mode.md b/docs/content/docs/dev/python/python_execution_mode.md index 2c96958108580..ef7afa9124e88 100644 --- a/docs/content/docs/dev/python/python_execution_mode.md +++ b/docs/content/docs/dev/python/python_execution_mode.md @@ -31,61 +31,48 @@ defines how to execute your customized Python functions. Prior to release-1.15, there is the only execution mode called `PROCESS` execution mode. The `PROCESS` mode means that the Python user-defined functions will be executed in separate Python processes. -In release-1.15, it has introduced another two execution modes called `MULTI-THREAD` execution mode and -`SUB-INTERPRETER` execution mode. The `MULTI-THREAD` mode means that the Python user-defined functions -will be executed in the same thread as Java Operator, but it will be affected by GIL performance. -The `SUB-INTERPRETER` mode means that the Python user-defined functions will be executed in Python -different sub-interpreters rather than different threads of one interpreter, which can largely overcome -the effects of the GIL, but some CPython extensions libraries doesn't support it, such as numpy, tensorflow, etc. +In release-1.15, it has introduced a new execution mode called `THREAD` execution mode. The `THREAD` +mode means that the Python user-defined functions will be executed in the same process as Java Operator, +It should be noted that multiple Python user-defined functions running in the same JVM are still affected by GIL. -## When can/should I use MULTI-THREAD execution mode or SUB-INTERPRETER execution mode? +## When can/should I use THREAD execution mode? -The purpose of the introduction of `MULTI-THREAD` mode and `SUB-INTERPRETER` mode is to overcome the -overhead of serialization/deserialization and network communication caused in `PROCESS` mode. -So if performance is not your concern, or the computing logic of your customized Python functions is -the performance bottleneck of the job, `PROCESS` mode will be the best choice as `PROCESS` mode provides -the best isolation compared to `MULTI-THREAD` mode and `SUB-INTERPRETER` mode. - -Compared to `MULTI-THREAD` execution mode, `SUB-INTERPRETER` execution mode can largely overcome the -effects of the GIL, so you can get better performance usually. However, `SUB-INTERPRETER` may fail in some CPython -extensions libraries, such as numpy, tensorflow. In this case, you should use `PROCESS` mode or `MULTI-THREAD` mode. +The purpose of the introduction of `THREAD` mode is to overcome the overhead of serialization/deserialization +and network communication caused in `PROCESS` mode. So if performance is not your concern, or the computing +logic of your customized Python functions is the performance bottleneck of the job, `PROCESS` mode will +be the best choice as `PROCESS` mode provides the best isolation compared to `THREAD` mode. ## Configuring Python execution mode The execution mode can be configured via the `python.execution-mode` setting. -There are three possible values: +There are two possible values: - `PROCESS`: The Python user-defined functions will be executed in separate Python process. (default) - - `MULTI-THREAD`: The Python user-defined functions will be executed in the same thread as Java Operator. - - `SUB-INTERPRETER`: The Python user-defined functions will be executed in Python different sub-interpreters. + - `THREAD`: The Python user-defined functions will be executed in the same process as Java operator. You could specify the Python execution mode using Python Table API as following: ```python # Specify `PROCESS` mode -table_env.get_config().get_configuration().set_string("python.execution-mode", "process") - -# Specify `MULTI-THREAD` mode -table_env.get_config().get_configuration().set_string("python.execution-mode", "multi-thread") +table_env.get_config().set("python.execution-mode", "process") -# Specify `SUB-INTERPRETER` mode -table_env.get_config().get_configuration().set_string("python.execution-mode", "sub-interpreter") +# Specify `THREAD` mode +table_env.get_config().set("python.execution-mode", "thread") ``` {{< hint info >}} -Currently, it still doesn't support to execute Python UDFs in `MULTI-THREAD` and `SUB-INTERPRETER` execution mode -in all places. It will fall back to `PROCESS` execution mode in these cases. So it may happen that you configure a job -to execute in `MULTI-THREAD` or `SUB-INTERPRETER` execution modes, however, it's actually executed in `PROCESS` execution mode. +Currently, it still doesn't support to execute Python UDFs in `THREAD` execution mode in all places. +It will fall back to `PROCESS` execution mode in these cases. So it may happen that you configure a job +to execute in `THREAD` execution mode, however, it's actually executed in `PROCESS` execution mode. {{< /hint >}} {{< hint info >}} -`MULTI-THREAD` execution mode only supports Python 3.7+. `SUB-INTERPRETER` execution mode only supports Python 3.8+. +`THREAD` execution mode is only supported in Python 3.7+. {{< /hint >}} ## Execution Behavior -This section provides an overview of the execution behavior of `MULTI-THREAD` and `SUB-INTERPRETER` -execution mode and contrasts they with `PROCESS` execution mode. For more -details, please refer to the FLIP that introduced this feature: +This section provides an overview of the execution behavior of `THREAD` execution mode and contrasts +they with `PROCESS` execution mode. For more details, please refer to the FLIP that introduced this feature: [FLIP-206](https://cwiki.apache.org/confluence/display/FLINK/FLIP-206%3A+Support+PyFlink+Runtime+Execution+in+Thread+Mode). #### PROCESS Execution Mode @@ -95,10 +82,10 @@ The Java operator process communicates with the Python worker process using vari {{< img src="/fig/pyflink_process_execution_mode.png" alt="Process Execution Mode" >}} -#### MULTI-THREAD and SUB-INTERPRETER Execution Mode +#### THREAD Execution Mode -In `MULTI-THREAD` and `SUB-INTERPRETER` execution mode, the Python user-defined functions will be executed in -the same process as Java operators. PyFlink takes use of third part library [PEMJA](https://github.com/alibaba/pemja) to -embed Python in Java Application. +In `THREAD` execution mode, the Python user-defined functions will be executed in the same process +as Java operators. PyFlink takes use of third part library [PEMJA](https://github.com/alibaba/pemja) +to embed Python in Java Application. {{< img src="/fig/pyflink_embedded_execution_mode.png" alt="Embedded Execution Mode" >}} 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 fdbea6c57fb8d..2b1e172b9f8ff 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 @@ -29,7 +29,7 @@ under the License. This document is a short introduction to the PyFlink Table API, which is used to help novice users quickly understand the basic usage of PyFlink Table API. For advanced usage, please refer to other documents in this user guide. -Common Structure of Python Table API Program +Common Structure of Python Table API Program -------------------------------------------- All Table API and SQL programs, both batch and streaming, follow the same pattern. The following code example shows the common structure of Table API and SQL programs. diff --git a/docs/content/docs/dev/python/table/python_table_api_connectors.md b/docs/content/docs/dev/python/table/python_table_api_connectors.md index 6b3743aa5aedf..a7f31d30cfb14 100644 --- a/docs/content/docs/dev/python/table/python_table_api_connectors.md +++ b/docs/content/docs/dev/python/table/python_table_api_connectors.md @@ -38,7 +38,7 @@ Since Flink is a Java/Scala-based project, for both connectors and formats, impl are available as jars that need to be specified as job [dependencies]({{< ref "docs/dev/python/dependency_management" >}}). ```python -table_env.get_config().get_configuration().set_string("pipeline.jars", "file:///my/jar/path/connector.jar;file:///my/jar/path/json.jar") +table_env.get_config().set("pipeline.jars", "file:///my/jar/path/connector.jar;file:///my/jar/path/json.jar") ``` ## How to use connectors @@ -89,7 +89,7 @@ def log_processing(): env_settings = EnvironmentSettings.in_streaming_mode() t_env = TableEnvironment.create(env_settings) # specify connector and format jars - t_env.get_config().get_configuration().set_string("pipeline.jars", "file:///my/jar/path/connector.jar;file:///my/jar/path/json.jar") + t_env.get_config().set("pipeline.jars", "file:///my/jar/path/connector.jar;file:///my/jar/path/json.jar") source_ddl = """ CREATE TABLE source_table( diff --git a/docs/content/docs/dev/python/table/table_environment.md b/docs/content/docs/dev/python/table/table_environment.md index 300d69a592479..5f2cfc61c9b2e 100644 --- a/docs/content/docs/dev/python/table/table_environment.md +++ b/docs/content/docs/dev/python/table/table_environment.md @@ -35,13 +35,18 @@ Create a TableEnvironment The recommended way to create a `TableEnvironment` is to create from an `EnvironmentSettings` object: ```python +from pyflink.common import Configuration from pyflink.table import EnvironmentSettings, TableEnvironment # create a streaming TableEnvironment -env_settings = EnvironmentSettings.in_streaming_mode() +config = Configuration() +config.set_string('execution.buffer-timeout', '1 min') +env_settings = EnvironmentSettings \ + .new_instance() \ + .in_streaming_mode() \ + .with_configuration(config) \ + .build() -# or a batch TableEnvironment -# env_settings = EnvironmentSettings.in_batch_mode() table_env = TableEnvironment.create(env_settings) ``` @@ -567,12 +572,9 @@ Please refer to the [Dependency Management]({{< ref "docs/dev/python/dependency_ Returns the table config to define the runtime behavior of the Table API. You can find all the available configuration options in }}">Configuration and }}">Python Configuration.

- The following code is an example showing how to set the configuration options through this API: -```python -# set the parallelism to 8 -table_env.get_config().get_configuration().set_string( - "parallelism.default", "8") -``` + The following code is an example showing how to set the configuration options through this API:
+# set the parallelism to 8
+table_env.get_config().set("parallelism.default", "8") {{< pythondoc file="pyflink.table.html#pyflink.table.TableEnvironment.get_config" name="link">}} @@ -821,19 +823,19 @@ And now you can configure them by setting key-value options in `TableConfig`, se The following code is an example showing how to configure the statebackend, checkpoint and restart strategy through the Table API: ```python # set the restart strategy to "fixed-delay" -table_env.get_config().get_configuration().set_string("restart-strategy", "fixed-delay") -table_env.get_config().get_configuration().set_string("restart-strategy.fixed-delay.attempts", "3") -table_env.get_config().get_configuration().set_string("restart-strategy.fixed-delay.delay", "30s") +table_env.get_config().set("restart-strategy", "fixed-delay") +table_env.get_config().set("restart-strategy.fixed-delay.attempts", "3") +table_env.get_config().set("restart-strategy.fixed-delay.delay", "30s") # set the checkpoint mode to EXACTLY_ONCE -table_env.get_config().get_configuration().set_string("execution.checkpointing.mode", "EXACTLY_ONCE") -table_env.get_config().get_configuration().set_string("execution.checkpointing.interval", "3min") +table_env.get_config().set("execution.checkpointing.mode", "EXACTLY_ONCE") +table_env.get_config().set("execution.checkpointing.interval", "3min") # set the statebackend type to "rocksdb", other available options are "filesystem" and "jobmanager" # you can also set the full qualified Java class name of the StateBackendFactory to this option # e.g. org.apache.flink.contrib.streaming.state.RocksDBStateBackendFactory -table_env.get_config().get_configuration().set_string("state.backend", "rocksdb") +table_env.get_config().set("state.backend", "rocksdb") # set the checkpoint directory, which is required by the RocksDB statebackend -table_env.get_config().get_configuration().set_string("state.checkpoints.dir", "file:///tmp/checkpoints/") +table_env.get_config().set("state.checkpoints.dir", "file:///tmp/checkpoints/") ``` diff --git a/docs/content/docs/dev/python/table_api_tutorial.md b/docs/content/docs/dev/python/table_api_tutorial.md index ae355bcb518c2..2e10ecde07201 100644 --- a/docs/content/docs/dev/python/table_api_tutorial.md +++ b/docs/content/docs/dev/python/table_api_tutorial.md @@ -70,7 +70,7 @@ The table config allows setting Table API specific configurations. ```python t_env = TableEnvironment.create(EnvironmentSettings.in_streaming_mode()) -t_env.get_config().get_configuration().set_string("parallelism.default", "1") +t_env.get_config().set("parallelism.default", "1") ``` You can now create the source and sink tables: @@ -204,7 +204,7 @@ word_count_data = ["To be, or not to be,--that is the question:--", def word_count(input_path, output_path): t_env = TableEnvironment.create(EnvironmentSettings.in_streaming_mode()) # write all the data to one file - t_env.get_config().get_configuration().set_string("parallelism.default", "1") + t_env.get_config().set("parallelism.default", "1") # define the source if input_path is not None: diff --git a/docs/content/docs/dev/table/catalogs.md b/docs/content/docs/dev/table/catalogs.md index 54b6941b51d3b..d0b7ab6be6416 100644 --- a/docs/content/docs/dev/table/catalogs.md +++ b/docs/content/docs/dev/table/catalogs.md @@ -76,7 +76,7 @@ Users can use SQL DDL to create tables in catalogs in both Table API and SQL. {{< tabs "b462513f-2da9-4bd0-a55d-ca9a5e4cf512" >}} {{< tab "Java" >}} ```java -TableEnvironment tableEnv = ... +TableEnvironment tableEnv = ...; // Create a HiveCatalog Catalog catalog = new HiveCatalog("myhive", null, ""); diff --git a/docs/content/docs/dev/table/common.md b/docs/content/docs/dev/table/common.md index 5a783ad5ae780..5fa3ead05099c 100644 --- a/docs/content/docs/dev/table/common.md +++ b/docs/content/docs/dev/table/common.md @@ -51,7 +51,7 @@ tableEnv.createTemporaryTable("SourceTable", TableDescriptor.forConnector("datag .column("f0", DataTypes.STRING()) .build()) .option(DataGenOptions.ROWS_PER_SECOND, 100) - .build()) + .build()); // Create a sink table (using SQL DDL) tableEnv.executeSql("CREATE TEMPORARY TABLE SinkTable WITH ('connector' = 'blackhole') LIKE SourceTable"); @@ -344,7 +344,7 @@ tableEnv.createTable("SourceTableA", sourceDescriptor); tableEnv.createTemporaryTable("SourceTableB", sourceDescriptor); // Using SQL DDL -tableEnv.executeSql("CREATE [TEMPORARY] TABLE MyTable (...) WITH (...)") +tableEnv.executeSql("CREATE [TEMPORARY] TABLE MyTable (...) WITH (...)"); ``` ### Expanding Table identifiers @@ -672,7 +672,7 @@ tableEnv.createTemporaryTable("CsvSinkTable", TableDescriptor.forConnector("file .build()); // compute a result Table using Table API operators and/or SQL queries -Table result = ... +Table result = ...; // Prepare the insert into pipeline TablePipeline pipeline = result.insertInto("CsvSinkTable"); @@ -921,10 +921,10 @@ tEnv.createTemporaryTable("MySink2", TableDescriptor.forConnector("filesystem") StatementSet stmtSet = tEnv.createStatementSet(); Table table1 = tEnv.from("MySource1").where($("word").like("F%")); -stmtSet.add(table1.insertInto("MySink1"); +stmtSet.add(table1.insertInto("MySink1")); Table table2 = table1.unionAll(tEnv.from("MySource2")); -stmtSet.add(table2.insertInto("MySink2"); +stmtSet.add(table2.insertInto("MySink2")); String explanation = stmtSet.explain(); System.out.println(explanation); diff --git a/docs/content/docs/dev/table/config.md b/docs/content/docs/dev/table/config.md index 9cc928d7e543d..6730703f1a996 100644 --- a/docs/content/docs/dev/table/config.md +++ b/docs/content/docs/dev/table/config.md @@ -35,7 +35,12 @@ that the required state size is capped (see [streaming concepts]({{< ref "docs/d ### Overview -In every table environment, the `TableConfig` offers options for configuring the current session. +When instantiating a `TableEnvironment`, `EnviromentSettings` can be used to pass the desired +configuration for the current session, by passing a `Configuration` object to the +`EnviromentSettings`. + +Additionally, in every table environment, the `TableConfig` offers options for configuring the +current session. For common or important configuration options, the `TableConfig` provides getters and setters methods with detailed inline documentation. @@ -51,40 +56,63 @@ table environment. {{< tab "Java" >}} ```java // instantiate table environment -TableEnvironment tEnv = ... - -// access flink configuration -TableConfig configuration = tEnv.getConfig(); +Configuration configuration = new Configuration(); // set low-level key-value options -configuration.set("table.exec.mini-batch.enabled", "true"); -configuration.set("table.exec.mini-batch.allow-latency", "5 s"); -configuration.set("table.exec.mini-batch.size", "5000"); +configuration.setString("table.exec.mini-batch.enabled", "true"); +configuration.setString("table.exec.mini-batch.allow-latency", "5 s"); +configuration.setString("table.exec.mini-batch.size", "5000"); +EnvironmentSettings settings = EnvironmentSettings.newInstance() + .inStreamingMode().withConfiguration(configuration).build(); +TableEnvironment tEnv = TableEnvironment.create(settings); + +// access flink configuration after table environment instantiation +TableConfig tableConfig = tEnv.getConfig(); +// set low-level key-value options +tableConfig.set("table.exec.mini-batch.enabled", "true"); +tableConfig.set("table.exec.mini-batch.allow-latency", "5 s"); +tableConfig.set("table.exec.mini-batch.size", "5000"); ``` {{< /tab >}} {{< tab "Scala" >}} ```scala // instantiate table environment -val tEnv: TableEnvironment = ... - -// access flink configuration -val configuration = tEnv.getConfig() +val configuration = new Configuration; // set low-level key-value options -configuration.set("table.exec.mini-batch.enabled", "true") -configuration.set("table.exec.mini-batch.allow-latency", "5 s") -configuration.set("table.exec.mini-batch.size", "5000") +configuration.setString("table.exec.mini-batch.enabled", "true") +configuration.setString("table.exec.mini-batch.allow-latency", "5 s") +configuration.setString("table.exec.mini-batch.size", "5000") +val settings = EnvironmentSettings.newInstance + .inStreamingMode.withConfiguration(configuration).build +val tEnv: TableEnvironment = TableEnvironment.create(settings) + +// access flink configuration after table environment instantiation +val tableConfig = tEnv.getConfig() +// set low-level key-value options +tableConfig.set("table.exec.mini-batch.enabled", "true") +tableConfig.set("table.exec.mini-batch.allow-latency", "5 s") +tableConfig.set("table.exec.mini-batch.size", "5000") ``` {{< /tab >}} {{< tab "Python" >}} ```python # instantiate table environment -t_env = ... - -# access flink configuration -configuration = t_env.get_config() -# set low-level key-value options +configuration = Configuration() configuration.set("table.exec.mini-batch.enabled", "true") configuration.set("table.exec.mini-batch.allow-latency", "5 s") configuration.set("table.exec.mini-batch.size", "5000") +settings = EnvironmentSettings.new_instance() \ +... .in_streaming_mode() \ +... .with_configuration(configuration) \ +... .build() + +t_env = TableEnvironment.create(settings) + +# access flink configuration after table environment instantiation +table_config = t_env.get_config() +# set low-level key-value options +table_config.set("table.exec.mini-batch.enabled", "true") +table_config.set("table.exec.mini-batch.allow-latency", "5 s") +table_config.set("table.exec.mini-batch.size", "5000") ``` {{< /tab >}} {{< tab "SQL CLI" >}} @@ -96,6 +124,13 @@ Flink SQL> SET 'table.exec.mini-batch.size' = '5000'; {{< /tab >}} {{< /tabs >}} +{{< hint info >}} +**Note:** All of the following configuration options can also be set globally in +`conf/flink-conf.yaml` (see [configuration]({{< ref "docs/deployment/config" >}}) and can be later +on overridden in the application, through `EnvironmentSettings`, before instantiating +the `TableEnvironment`, or through the `TableConfig` of the `TableEnvironment`. +{{< /hint >}} + ### Execution Options The following options can be used to tune the performance of the query execution. diff --git a/docs/content/docs/dev/table/data_stream_api.md b/docs/content/docs/dev/table/data_stream_api.md index 7506149da9702..28950e2e942db 100644 --- a/docs/content/docs/dev/table/data_stream_api.md +++ b/docs/content/docs/dev/table/data_stream_api.md @@ -598,25 +598,25 @@ pipeline or a statement set: ```java // execute with explicit sink -tableEnv.from("InputTable").insertInto("OutputTable").execute() +tableEnv.from("InputTable").insertInto("OutputTable").execute(); -tableEnv.executeSql("INSERT INTO OutputTable SELECT * FROM InputTable") +tableEnv.executeSql("INSERT INTO OutputTable SELECT * FROM InputTable"); tableEnv.createStatementSet() .add(tableEnv.from("InputTable").insertInto("OutputTable")) .add(tableEnv.from("InputTable").insertInto("OutputTable2")) - .execute() + .execute(); tableEnv.createStatementSet() .addInsertSql("INSERT INTO OutputTable SELECT * FROM InputTable") .addInsertSql("INSERT INTO OutputTable2 SELECT * FROM InputTable") - .execute() + .execute(); // execute with implicit local sink -tableEnv.from("InputTable").execute().print() +tableEnv.from("InputTable").execute().print(); -tableEnv.executeSql("SELECT * FROM InputTable").print() +tableEnv.executeSql("SELECT * FROM InputTable").print(); ``` To combine both execution behaviors, every call to `StreamTableEnvironment.toDataStream` @@ -629,17 +629,17 @@ these "external parts". // (1) // adds a branch with a printing sink to the StreamExecutionEnvironment -tableEnv.toDataStream(table).print() +tableEnv.toDataStream(table).print(); // (2) // executes a Table API end-to-end pipeline as a Flink job and prints locally, // thus (1) has still not been executed -table.execute().print() +table.execute().print(); // executes the DataStream API pipeline with the sink defined in (1) as a // Flink job, (2) was already running before -env.execute() +env.execute(); ``` {{< top >}} @@ -2771,7 +2771,7 @@ The schema of the resulting view depends on the data type of the registered coll {{< tab "Java" >}} ```java StreamTableEnvironment tableEnv = ...; -DataStream> stream = ... +DataStream> stream = ...; Table table2 = tableEnv.fromDataStream(stream, $("myLong"), $("myString")); ``` @@ -2926,7 +2926,7 @@ When defining a position-based mapping, the specified names must not exist in th ```java StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section; -DataStream> stream = ... +DataStream> stream = ...; // convert DataStream into Table with field "myLong" only Table table = tableEnv.fromDataStream(stream, $("myLong")); @@ -2978,7 +2978,7 @@ If no field names are specified, the default field names and field order of the ```java StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section -DataStream> stream = ... +DataStream> stream = ...; // convert DataStream into Table with field "f1" only Table table = tableEnv.fromDataStream(stream, $("f1")); @@ -3039,7 +3039,7 @@ The type of the column is inferred from the atomic type. The name of the column ```java StreamTableEnvironment tableEnv = ...; -DataStream stream = ... +DataStream stream = ...; // Convert DataStream into Table with field name "myLong" Table table = tableEnv.fromDataStream(stream, $("myLong")); @@ -3089,7 +3089,7 @@ Name-based mapping allows for reordering fields and projection with alias (`as`) ```java StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section -DataStream> stream = ... +DataStream> stream = ...; // convert DataStream into Table with renamed field names "myLong", "myString" (position-based) Table table = tableEnv.fromDataStream(stream, $("myLong"), $("myString")); @@ -3183,7 +3183,7 @@ When converting a POJO `DataStream` into a `Table` without specifying field name StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section // Person is a POJO with fields "name" and "age" -DataStream stream = ... +DataStream stream = ...; // convert DataStream into Table with renamed fields "myAge", "myName" (name-based) Table table = tableEnv.fromDataStream(stream, $("age").as("myAge"), $("name").as("myName")); @@ -3230,7 +3230,7 @@ Fields can be renamed by providing names for all fields (mapping based on positi StreamTableEnvironment tableEnv = ...; // DataStream of Row with two fields "name" and "age" specified in `RowTypeInfo` -DataStream stream = ... +DataStream stream = ...; // Convert DataStream into Table with renamed field names "myName", "myAge" (position-based) Table table = tableEnv.fromDataStream(stream, $("myName"), $("myAge")); diff --git a/docs/content/docs/dev/table/functions/systemFunctions.md b/docs/content/docs/dev/table/functions/systemFunctions.md index a07cfa208e06d..9e88fdbe9c5f5 100644 --- a/docs/content/docs/dev/table/functions/systemFunctions.md +++ b/docs/content/docs/dev/table/functions/systemFunctions.md @@ -209,7 +209,7 @@ The column functions can be used in all places where column fields are expected, ```java table .groupBy("withColumns(1 to 3)") - .select("withColumns(a to b), myUDAgg(myUDF(withColumns(5 to 20)))") + .select("withColumns(a to b), myUDAgg(myUDF(withColumns(5 to 20)))"); ``` {{< /tab >}} {{< tab "Scala" >}} diff --git a/docs/content/docs/dev/table/tuning.md b/docs/content/docs/dev/table/tuning.md index 804e75f7d6661..2b97e6afb621f 100644 --- a/docs/content/docs/dev/table/tuning.md +++ b/docs/content/docs/dev/table/tuning.md @@ -59,7 +59,7 @@ The following examples show how to enable these options. {{< tab "Java" >}} ```java // instantiate table environment -TableEnvironment tEnv = ... +TableEnvironment tEnv = ...; // access flink configuration TableConfig configuration = tEnv.getConfig(); @@ -88,11 +88,11 @@ configuration.set("table.exec.mini-batch.size", "5000") // the maximum number of t_env = ... # access flink configuration -configuration = t_env.get_config(); +configuration = t_env.get_config() # set low-level key-value options -configuration.set("table.exec.mini-batch.enabled", "true"); # enable mini-batch optimization -configuration.set("table.exec.mini-batch.allow-latency", "5 s"); # use 5 seconds to buffer input records -configuration.set("table.exec.mini-batch.size", "5000"); # the maximum number of records can be buffered by each aggregate operator task +configuration.set("table.exec.mini-batch.enabled", "true") # enable mini-batch optimization +configuration.set("table.exec.mini-batch.allow-latency", "5 s") # use 5 seconds to buffer input records +configuration.set("table.exec.mini-batch.size", "5000") # the maximum number of records can be buffered by each aggregate operator task ``` {{< /tab >}} {{< /tabs >}} @@ -122,7 +122,7 @@ The following examples show how to enable the local-global aggregation. {{< tab "Java" >}} ```java // instantiate table environment -TableEnvironment tEnv = ... +TableEnvironment tEnv = ...; // access flink configuration TableConfig configuration = tEnv.getConfig(); @@ -153,12 +153,12 @@ configuration.set("table.optimizer.agg-phase-strategy", "TWO_PHASE") // enable t t_env = ... # access flink configuration -configuration = t_env.get_config(); +configuration = t_env.get_config() # set low-level key-value options -configuration.set("table.exec.mini-batch.enabled", "true"); # local-global aggregation depends on mini-batch is enabled -configuration.set("table.exec.mini-batch.allow-latency", "5 s"); -configuration.set("table.exec.mini-batch.size", "5000"); -configuration.set("table.optimizer.agg-phase-strategy", "TWO_PHASE"); # enable two-phase, i.e. local-global aggregation +configuration.set("table.exec.mini-batch.enabled", "true") # local-global aggregation depends on mini-batch is enabled +configuration.set("table.exec.mini-batch.allow-latency", "5 s") +configuration.set("table.exec.mini-batch.size", "5000") +configuration.set("table.optimizer.agg-phase-strategy", "TWO_PHASE") # enable two-phase, i.e. local-global aggregation ``` {{< /tab >}} {{< /tabs >}} @@ -210,7 +210,7 @@ The following examples show how to enable the split distinct aggregation optimiz {{< tab "Java" >}} ```java // instantiate table environment -TableEnvironment tEnv = ... +TableEnvironment tEnv = ...; tEnv.getConfig() .set("table.optimizer.distinct-agg.split.enabled", "true"); // enable distinct agg split @@ -230,8 +230,7 @@ tEnv.getConfig # instantiate table environment t_env = ... -t_env.get_config() - .set("table.optimizer.distinct-agg.split.enabled", "true"); # enable distinct agg split +t_env.get_config().set("table.optimizer.distinct-agg.split.enabled", "true") # enable distinct agg split ``` {{< /tab >}} {{< /tabs >}} diff --git a/docs/content/docs/dev/table/types.md b/docs/content/docs/dev/table/types.md index 27d4c2860d3f5..9d3fb63c9e65d 100644 --- a/docs/content/docs/dev/table/types.md +++ b/docs/content/docs/dev/table/types.md @@ -1553,7 +1553,7 @@ regardless of whether the function used is `CAST` or `TRY_CAST`. ### Legacy casting Pre Flink 1.15 casting behaviour can be enabled by setting `table.exec.legacy-cast-behaviour` to `enabled`. -In Flink 1.15 this flag is enabled by default. +In Flink 1.15 this flag is disabled by default. In particular, this will: @@ -1562,7 +1562,7 @@ In particular, this will: * Formatting of some casting to `CHAR`/`VARCHAR`/`STRING` produces slightly different results. {{< hint warning >}} -We **discourage** the use of this flag and we **strongly suggest** for new projects to disable this flag and use the new casting behaviour. +We **discourage** the use of this flag and we **strongly suggest** for new projects to keep this flag disabled and use the new casting behaviour. This flag will be removed in the next Flink versions. {{< /hint >}} diff --git a/docs/content/docs/learn-flink/streaming_analytics.md b/docs/content/docs/learn-flink/streaming_analytics.md index b70c675e7dc0f..f3ddc1393a83c 100644 --- a/docs/content/docs/learn-flink/streaming_analytics.md +++ b/docs/content/docs/learn-flink/streaming_analytics.md @@ -134,7 +134,7 @@ a class that extracts the timestamps from the events, and generates watermarks o easiest way to do this is by using a `WatermarkStrategy`: ```java -DataStream stream = ... +DataStream stream = ...; WatermarkStrategy strategy = WatermarkStrategy .forBoundedOutOfOrderness(Duration.ofSeconds(20)) @@ -244,7 +244,7 @@ end-of-window-timestamp, max_value)`. #### ProcessWindowFunction Example ```java -DataStream input = ... +DataStream input = ...; input .keyBy(x -> x.key) @@ -296,7 +296,7 @@ per-key information for all windows of that key. This might be useful, for examp #### Incremental Aggregation Example ```java -DataStream input = ... +DataStream input = ...; input .keyBy(x -> x.key) diff --git a/docs/content/docs/libs/cep.md b/docs/content/docs/libs/cep.md index b927cdb67dd3c..fc541e2afd946 100644 --- a/docs/content/docs/libs/cep.md +++ b/docs/content/docs/libs/cep.md @@ -65,7 +65,7 @@ because FlinkCEP uses them for comparing and matching events. {{< tabs "8951ef0a-cdd4-40d1-bda8-dec1299aaf41" >}} {{< tab "Java" >}} ```java -DataStream input = ... +DataStream input = ...; Pattern pattern = Pattern.begin("start").where( new SimpleCondition() { @@ -341,7 +341,7 @@ via the `pattern.subtype(subClass)` method. start.subtype(SubEvent.class).where(new SimpleCondition() { @Override public boolean filter(SubEvent value) { - return ... // some condition + return ...; // some condition } }); ``` @@ -361,12 +361,12 @@ start.subtype(classOf[SubEvent]).where(subEvent => ... /* some condition */) pattern.where(new SimpleCondition() { @Override public boolean filter(Event value) { - return ... // some condition + return ...; // some condition } }).or(new SimpleCondition() { @Override public boolean filter(Event value) { - return ... // or condition + return ...; // or condition } }); ``` @@ -403,7 +403,7 @@ Multiple consecutive where() clauses lead to their conditions being `AND`ed. pattern.where(new IterativeCondition() { @Override public boolean filter(Event value, Context ctx) throws Exception { - return ... // some condition + return ...; // some condition } }); ``` @@ -425,12 +425,12 @@ Adds a new condition which is `OR`ed with an existing one. An event can match th pattern.where(new IterativeCondition() { @Override public boolean filter(Event value, Context ctx) throws Exception { - return ... // some condition + return ...; // some condition } }).or(new IterativeCondition() { @Override public boolean filter(Event value, Context ctx) throws Exception { - return ... // alternative condition + return ...; // alternative condition } }); ``` @@ -455,7 +455,7 @@ events will be accepted into the pattern. Applicable only in conjunction with `o pattern.oneOrMore().until(new IterativeCondition() { @Override public boolean filter(Event value, Context ctx) throws Exception { - return ... // alternative condition + return ...; // alternative condition } }); ``` @@ -1248,7 +1248,7 @@ Then apply the skip strategy to a pattern by calling: {{< tabs "64a34dcc-47f8-443d-b31a-515f7fd17243" >}} {{< tab "Java" >}} ```java -AfterMatchSkipStrategy skipStrategy = ... +AfterMatchSkipStrategy skipStrategy = ...; Pattern.begin("patternName", skipStrategy); ``` {{< /tab >}} @@ -1269,7 +1269,7 @@ One can enable this option by: {{< tabs "59e07b27-61d3-4348-ab60-c8a805500c87" >}} {{< tab "Java" >}} ```java -AfterMatchSkipStrategy.skipToFirst(patternName).throwExceptionOnMiss() +AfterMatchSkipStrategy.skipToFirst(patternName).throwExceptionOnMiss(); ``` {{< /tab >}} {{< tab "Scala" >}} @@ -1288,9 +1288,9 @@ Given an input stream `input`, a pattern `pattern` and an optional comparator `c {{< tabs "79719c8a-f503-4f3e-9717-75540e637481" >}} {{< tab "Java" >}} ```java -DataStream input = ... -Pattern pattern = ... -EventComparator comparator = ... // optional +DataStream input = ...; +Pattern pattern = ...; +EventComparator comparator = ...; // optional PatternStream patternStream = CEP.pattern(input, pattern, comparator); ``` @@ -1528,9 +1528,9 @@ The whole processing is done with event time. {{< tabs "573ac3c5-e8b9-4ffa-b7b6-e2db19611ff5" >}} {{< tab "Java" >}} ```java -StreamExecutionEnvironment env = ... +StreamExecutionEnvironment env = ...; -DataStream input = ... +DataStream input = ...; DataStream partitionedInput = input.keyBy(new KeySelector() { @Override diff --git a/docs/content/docs/libs/gelly/bipartite_graph.md b/docs/content/docs/libs/gelly/bipartite_graph.md index aefc527d3572e..e2763ebe932c0 100644 --- a/docs/content/docs/libs/gelly/bipartite_graph.md +++ b/docs/content/docs/libs/gelly/bipartite_graph.md @@ -84,11 +84,11 @@ You can create a `BipartiteGraph` in the following ways: ```java ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); -DataSet> topVertices = ... +DataSet> topVertices = ...; -DataSet> bottomVertices = ... +DataSet> bottomVertices = ...; -DataSet> edges = ... +DataSet> edges = ...; Graph graph = BipartiteGraph.fromDataSet(topVertices, bottomVertices, edges, env); ``` @@ -120,14 +120,14 @@ In the case of a simple projection each node in the result graph contains a pair ```java ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); // Vertices (1, "top1") -DataSet> topVertices = ... +DataSet> topVertices = ...; // Vertices (2, "bottom2"); (4, "bottom4") -DataSet> bottomVertices = ... +DataSet> bottomVertices = ...; // Edge that connect vertex 2 to vertex 1 and vertex 4 to vertex 1: // (1, 2, "1-2-edge"); (1, 4, "1-4-edge") -DataSet> edges = ... +DataSet> edges = ...; BipartiteGraph graph = BipartiteGraph.fromDataSet(topVertices, bottomVertices, edges, env); @@ -155,14 +155,14 @@ Full projection preserves all the information about the connection between two v ```java ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); // Vertices (1, "top1") -DataSet> topVertices = ... +DataSet> topVertices = ...; // Vertices (2, "bottom2"); (4, "bottom4") -DataSet> bottomVertices = ... +DataSet> bottomVertices = ...; // Edge that connect vertex 2 to vertex 1 and vertex 4 to vertex 1: // (1, 2, "1-2-edge"); (1, 4, "1-4-edge") -DataSet> edges = ... +DataSet> edges = ...; BipartiteGraph graph = BipartiteGraph.fromDataSet(topVertices, bottomVertices, edges, env); diff --git a/docs/content/docs/libs/gelly/graph_api.md b/docs/content/docs/libs/gelly/graph_api.md index a488a56125ff8..bd3f6b24c672b 100644 --- a/docs/content/docs/libs/gelly/graph_api.md +++ b/docs/content/docs/libs/gelly/graph_api.md @@ -96,9 +96,9 @@ You can create a `Graph` in the following ways: ```java ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); -DataSet> vertices = ... +DataSet> vertices = ...; -DataSet> edges = ... +DataSet> edges = ...; Graph graph = Graph.fromDataSet(vertices, edges, env); ``` @@ -123,7 +123,7 @@ val graph = Graph.fromDataSet(vertices, edges, env) ```java ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); -DataSet> edges = ... +DataSet> edges = ...; Graph graph = Graph.fromTuple2DataSet(edges, env); ``` @@ -231,9 +231,9 @@ val simpleGraph = Graph.fromCsvReader[Long, Double, NullValue]( ```java ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); -List> vertexList = new ArrayList... +List> vertexList = new ArrayList...; -List> edgeList = new ArrayList... +List> edgeList = new ArrayList...; Graph graph = Graph.fromCollection(vertexList, edgeList, env); ``` @@ -408,7 +408,7 @@ Graph updatedGraph = graph.translateGraphIds( Graph updatedGraph = graph .translateGraphIds(new LongToLongValue()) .translateVertexValues(new LongToLongValue()) - .translateEdgeValues(new LongToLongValue()) + .translateEdgeValues(new LongToLongValue()); ``` {{< /tab >}} {{< tab "Scala" >}} @@ -428,7 +428,7 @@ val updatedGraph = graph.translateGraphIds(id => id.toString) {{< tabs "b33fe8f8-8a53-4710-9379-8d2f912a3105" >}} {{< tab "Java" >}} ```java -Graph graph = ... +Graph graph = ...; graph.subgraph( new FilterFunction>() { @@ -467,7 +467,7 @@ Note that if the input dataset contains a key multiple times, all Gelly join met {{< tabs "219b4d15-4be2-4bbf-a3ea-4155d3f6ba27" >}} {{< tab "Java" >}} ```java -Graph network = ... +Graph network = ...; DataSet> vertexOutDegrees = network.outDegrees(); @@ -519,11 +519,11 @@ val networkWithWeights = network.joinWithEdgesOnSource(vertexOutDegrees, (v1: Do ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); // create first graph from edges {(1, 3, 12) (1, 3, 13), (1, 3, 13)} -List> edges1 = ... +List> edges1 = ...; Graph graph1 = Graph.fromCollection(edges1, env); // create second graph from edges {(1, 3, 13)} -List> edges2 = ... +List> edges2 = ...; Graph graph2 = Graph.fromCollection(edges2, env); // Using distinct = true results in {(1,3,13)} @@ -638,7 +638,7 @@ The following code will collect the out-edges for each vertex and apply the `Sel {{< tabs "8ab0141f-ed3d-4372-bfab-7f78ed6d7d5f" >}} {{< tab "Java" >}} ```java -Graph graph = ... +Graph graph = ...; DataSet> minWeights = graph.reduceOnEdges(new SelectMinWeight(), EdgeDirection.OUT); @@ -677,7 +677,7 @@ Similarly, assume that you would like to compute the sum of the values of all in {{< tabs "67e6fe66-aef8-46b8-8e80-2762dd5c3f02" >}} {{< tab "Java" >}} ```java -Graph graph = ... +Graph graph = ...; DataSet> verticesWithSum = graph.reduceOnNeighbors(new SumValues(), EdgeDirection.IN); @@ -720,7 +720,7 @@ For example, the following code will output all the vertex pairs which are conne {{< tabs "2cf7a021-b67a-42dc-912f-ef79f36314b2" >}} {{< tab "Java" >}} ```java -Graph graph = ... +Graph graph = ...; DataSet, Vertex>> vertexPairs = graph.groupReduceOnNeighbors(new SelectLargeWeightNeighbors(), EdgeDirection.OUT); @@ -783,10 +783,10 @@ also exist in the vertex IDs set. ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); // create a list of vertices with IDs = {1, 2, 3, 4, 5} -List> vertices = ... +List> vertices = ...; // create a list of edges with IDs = {(1, 2) (1, 3), (2, 4), (5, 6)} -List> edges = ... +List> edges = ...; Graph graph = Graph.fromCollection(vertices, edges, env); diff --git a/docs/content/docs/libs/gelly/graph_generators.md b/docs/content/docs/libs/gelly/graph_generators.md index ab5afe8843007..0ea4cf0a7ae5e 100644 --- a/docs/content/docs/libs/gelly/graph_generators.md +++ b/docs/content/docs/libs/gelly/graph_generators.md @@ -304,7 +304,7 @@ two `endpoint` vertices with degree `1` and all midpoint vertices with degree ```java ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); -long vertexCount = 5 +long vertexCount = 5; Graph graph = new PathGraph(env, vertexCount) .generate(); @@ -417,7 +417,7 @@ An undirected graph containing isolated two-paths where every vertex has degree ```java ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); -long vertexPairCount = 4 +long vertexPairCount = 4; // note: configured with the number of vertex pairs Graph graph = new SingletonEdgeGraph(env, vertexPairCount) diff --git a/docs/content/docs/libs/gelly/iterative_graph_processing.md b/docs/content/docs/libs/gelly/iterative_graph_processing.md index 442dab34e6672..1180a58a5058a 100644 --- a/docs/content/docs/libs/gelly/iterative_graph_processing.md +++ b/docs/content/docs/libs/gelly/iterative_graph_processing.md @@ -50,7 +50,7 @@ Let us consider computing Single-Source-Shortest-Paths with vertex-centric itera {{< tab "Java" >}} ```java // read the input graph -Graph graph = ... +Graph graph = ...; // define the maximum number of iterations int maxIterations = 10; @@ -178,7 +178,7 @@ all aggregates globally once per superstep and makes them available in the next {{< tab "Java" >}} ```java -Graph graph = ... +Graph graph = ...; // configure the iteration VertexCentricConfiguration parameters = new VertexCentricConfiguration(); @@ -296,7 +296,7 @@ Let us consider computing Single-Source-Shortest-Paths with scatter-gather itera {{< tab "Java" >}} ```java // read the input graph -Graph graph = ... +Graph graph = ...; // define the maximum number of iterations int maxIterations = 10; @@ -421,7 +421,7 @@ If the degrees option is not set in the configuration, these methods will return {{< tab "Java" >}} ```java -Graph graph = ... +Graph graph = ...; // configure the iteration ScatterGatherConfiguration parameters = new ScatterGatherConfiguration(); @@ -525,7 +525,7 @@ The following example illustrates the usage of the degree as well as the number {{< tab "Java" >}} ```java -Graph graph = ... +Graph graph = ...; // configure the iteration ScatterGatherConfiguration parameters = new ScatterGatherConfiguration(); @@ -599,7 +599,7 @@ The following example illustrates the usage of the edge direction option. Vertic {{< tabs "5efb0e23-7fd4-4e08-9952-981d6e4f3b9e" >}} {{< tab "Java" >}} ```java -Graph, Double> graph = ... +Graph, Double> graph = ...; // configure the iteration ScatterGatherConfiguration parameters = new ScatterGatherConfiguration(); @@ -664,7 +664,7 @@ To implement this example in Gelly GSA, the user only needs to call the `runGath {{< tab "Java" >}} ```java // read the input graph -Graph graph = ... +Graph graph = ...; // define the maximum number of iterations int maxIterations = 10; @@ -786,7 +786,7 @@ The following example illustrates the usage of the number of vertices option. {{< tab "Java" >}} ```java -Graph graph = ... +Graph graph = ...; // configure the iteration GSAConfiguration parameters = new GSAConfiguration(); @@ -868,7 +868,7 @@ The following example illustrates the usage of the edge direction option. {{< tab "Java" >}} ```java -Graph, Double> graph = ... +Graph, Double> graph = ...; // configure the iteration GSAConfiguration parameters = new GSAConfiguration(); diff --git a/docs/content/docs/libs/gelly/library_methods.md b/docs/content/docs/libs/gelly/library_methods.md index c83650fdd8c28..6976141f679ed 100644 --- a/docs/content/docs/libs/gelly/library_methods.md +++ b/docs/content/docs/libs/gelly/library_methods.md @@ -35,7 +35,7 @@ Gelly's library methods can be used by simply calling the `run()` method on the ```java ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); -Graph graph = ... +Graph graph = ...; // run Label Propagation for 30 iterations to detect communities on the input graph DataSet> verticesWithCommunity = graph.run(new LabelPropagation(30)); diff --git a/docs/content/docs/libs/state_processor_api.md b/docs/content/docs/libs/state_processor_api.md index f3be14e5b7c99..82c9fb8952cda 100644 --- a/docs/content/docs/libs/state_processor_api.md +++ b/docs/content/docs/libs/state_processor_api.md @@ -261,7 +261,7 @@ class ClickCounter implements AggregateFunction { } } -DataStream clicks = . . . +DataStream clicks = ...; clicks .keyBy(click -> click.userId) diff --git a/docs/content/docs/ops/metrics.md b/docs/content/docs/ops/metrics.md index a06f62a0a710b..3a574e3999588 100644 --- a/docs/content/docs/ops/metrics.md +++ b/docs/content/docs/ops/metrics.md @@ -1303,7 +1303,7 @@ Note that the metrics are only available via reporters. uploadQueueSize Current size of upload queue. Queue items can be packed together and form a single upload. - Meter + Gauge diff --git a/docs/content/docs/ops/state/checkpoints.md b/docs/content/docs/ops/state/checkpoints.md index 63f0c0694dfb7..2d4f6ecd6a893 100644 --- a/docs/content/docs/ops/state/checkpoints.md +++ b/docs/content/docs/ops/state/checkpoints.md @@ -35,6 +35,8 @@ the same semantics as a failure-free execution. See [Checkpointing]({{< ref "docs/dev/datastream/fault-tolerance/checkpointing" >}}) for how to enable and configure checkpoints for your program. +To understand the differences between checkpoints and [savepoints]({{< ref "docs/ops/state/savepoints" >}}) see [checkpoints vs. savepoints]({{< ref "docs/ops/state/checkpoints_vs_savepoints" >}}). + ## Checkpoint Storage When checkpointing is enabled, managed state is persisted to ensure consistent recovery in case of failures. @@ -156,12 +158,6 @@ env.getCheckpointConfig().setCheckpointStorage( new FileSystemCheckpointStorage("hdfs:///checkpoints-data/", FILE_SIZE_THESHOLD)); ``` -### Difference to Savepoints - -Checkpoints have a few differences from [savepoints]({{< ref "docs/ops/state/savepoints" >}}). They -- use a state backend specific (low-level) data format, may be incremental. (starting from Flink 1.15 savepoints can also use the backend [native]({{< ref "docs/ops/state/savepoints" >}}#savepoint-format) format.) -- do not support Flink specific features like rescaling. - ### Resuming from a retained checkpoint A job may be resumed from a checkpoint just as from a savepoint @@ -175,6 +171,4 @@ above). $ bin/flink run -s :checkpointMetaDataPath [:runArgs] ``` - - {{< top >}} diff --git a/docs/content/docs/ops/state/checkpoints_vs_savepoints.md b/docs/content/docs/ops/state/checkpoints_vs_savepoints.md new file mode 100644 index 0000000000000..0b66b256ea9ff --- /dev/null +++ b/docs/content/docs/ops/state/checkpoints_vs_savepoints.md @@ -0,0 +1,95 @@ +--- +title: "Checkpoints vs. Savepoints" +weight: 10 +type: docs +aliases: + - /ops/state/checkpoints_vs_savepoints.html +--- + + +# Checkpoints vs. Savepoints + +## Overview + +Conceptually, Flink's [savepoints]({{< ref "docs/ops/state/savepoints" >}}) are different from [checkpoints]({{< ref "docs/ops/state/checkpoints" >}}) +in a way that's analogous to how backups are different from recovery logs in traditional database systems. + +The primary purpose of checkpoints is to provide a recovery mechanism in case of unexpected job failures. +A [checkpoint's lifecycle]({{< ref "docs/dev/datastream/fault-tolerance/checkpointing" >}}) is managed by Flink, +i.e. a checkpoint is created, owned, and released by Flink - without user interaction. +Because checkpoints are being triggered often, and are relied upon for failure recovery, the two main design goals for the checkpoint implementation are +i) being as lightweight to create and ii) being as fast to restore from as possible. +Optimizations towards those goals can exploit certain properties, e.g., that the job code doesn't change between the execution attempts. + +{{< hint info >}} +- Checkpoints are automatically deleted if the application is terminated by the user +(except if checkpoints are explicitly configured to be retained). +- Checkpoints are stored in state backend-specific (native) data format (may be incremental depending on the specific backend). +{{< /hint >}} + +Although [savepoints]({{< ref "docs/ops/state/savepoints" >}}) are created internally with the same mechanisms as +checkpoints, they are conceptually different and can be a bit more expensive to produce and restore from. Their design focuses +more on portability and operational flexibility, especially with respect to changes to the job. +The use case for savepoints is for planned, manual operations. For example, this could be an update of your Flink version, changing your job graph, and so on. + +{{< hint info >}} +- Savepoints are created, owned and deleted solely by the user. +That means, Flink does not delete savepoints neither after job termination nor after +restore. +- Savepoints are stored in a state backend independent (canonical) format (Note: Since Flink 1.15, savepoints can be also stored in +the backend-specific [native]({{< ref "docs/ops/state/savepoints" >}}#savepoint-format) format which is faster to create +and restore but comes with some limitations. +{{< /hint >}} + +### Capabilities and limitations +The following table gives an overview of capabilities and limitations for the various types of savepoints and +checkpoints. +- ✓ - Flink fully support this type of the snapshot +- x - Flink doesn't support this type of the snapshot +- ! - While these operations currently work, Flink doesn't officially guarantee support for them, so there is a certain level of risk associated with them + +| Operation | Canonical Savepoint | Native Savepoint | Aligned Checkpoint | Unaligned Checkpoint | +|:----------------------------------|:--------------------|:-----------------|:-------------------|:---------------------| +| State backend change | ✓ | x | x | x | +| State Processor API (writing) | ✓ | x | x | x | +| State Processor API (reading) | ✓ | ! | ! | x | +| Self-contained and relocatable | ✓ | ✓ | x | x | +| Schema evolution | ✓ | ! | ! | ! | +| Arbitrary job upgrade | ✓ | ✓ | ✓ | x | +| Non-arbitrary job upgrade | ✓ | ✓ | ✓ | x | +| Flink minor version upgrade | ✓ | ✓ | ✓ | x | +| Flink bug/patch version upgrade | ✓ | ✓ | ✓ | ✓ | +| Rescaling | ✓ | ✓ | ✓ | ✓ | + +- [State backend change]({{< ref "docs/ops/state/state_backends" >}}) - configuring a different State Backend than was used when taking the snapshot. +- [State Processor API (writing)]({{< ref "docs/libs/state_processor_api" >}}#writing-new-savepoints) - the ability to create a new snapshot of this type via the State Processor API. +- [State Processor API (reading)]({{< ref "docs/libs/state_processor_api" >}}#reading-state) - the ability to read states from an existing snapshot of this type via the State Processor API. +- Self-contained and relocatable - the one snapshot folder contains everything it needs for recovery +and it doesn't depend on other snapshots which means it can be easily moved to another place if needed. +- [Schema evolution]({{< ref "docs/dev/datastream/fault-tolerance/serialization/schema_evolution" >}}) - the *state* data type can be changed if it uses a serializer that supports schema evolution (e.g., POJOs and Avro types) +- Arbitrary job upgrade - the snapshot can be restored even if the [partitioning types]({{< ref "docs/dev/datastream/operators/overview" >}}#physical-partitioning)(rescale, rebalance, map, etc.) +or in-flight record types for the existing operators have changed. +- Non-arbitrary job upgrade - restoring the snapshot is possible with updated operators if the job graph topology and in-flight record types remain unchanged. +- Flink minor version upgrade - restoring a snapshot taken with an older minor version of Flink (1.x → 1.y). +- Flink bug/patch version upgrade - restoring a snapshot taken with an older patch version of Flink (1.14.x → 1.14.y). +- Rescaling - restoring the snapshot with a different parallelism than was used during the snapshot creation. + + +{{< top >}} diff --git a/docs/content/docs/ops/state/savepoints.md b/docs/content/docs/ops/state/savepoints.md index 3058b83da5b53..2b182e43e6c12 100644 --- a/docs/content/docs/ops/state/savepoints.md +++ b/docs/content/docs/ops/state/savepoints.md @@ -27,7 +27,7 @@ under the License. # Savepoints -## What is a Savepoint? How is a Savepoint different from a Checkpoint? +## What is a Savepoint? A Savepoint is a consistent image of the execution state of a streaming job, created via Flink's [checkpointing mechanism]({{< ref "docs/learn-flink/fault_tolerance" >}}). You can use Savepoints to stop-and-resume, fork, or update your Flink jobs. Savepoints consist of two parts: a directory with (typically large) binary files on stable storage (e.g. HDFS, S3, ...) and a (relatively small) meta data file. The files on stable storage represent the net data of the job's execution state @@ -37,14 +37,7 @@ image. The meta data file of a Savepoint contains (primarily) pointers to all fi In order to allow upgrades between programs and Flink versions, it is important to check out the following section about [assigning IDs to your operators](#assigning-operator-ids). {{< /hint >}} -Conceptually, Flink's Savepoints are different from Checkpoints in a similar way that backups are different from recovery logs in traditional database systems. The primary purpose of Checkpoints is to provide a recovery mechanism in case of -unexpected job failures. A Checkpoint's lifecycle is managed by Flink, i.e. a Checkpoint is created, owned, and deleted by Flink - without user interaction. As a method of recovery and being periodically triggered, two main -design goals for the Checkpoint implementation are i) being as lightweight to create and ii) being as fast to restore from as possible. Optimizations towards those goals can exploit certain properties, e.g. that the job code -does not change between the execution attempts. Checkpoints are usually dropped after the job was terminated by the user unless the job is explicitly configured to retain checkpoints upon failure or cancellation. - -In contrast to all this, Savepoints are created, owned, and deleted by the user. Their use case is planned, manual backup and resume. For example, this could be an update of your Flink version, changing your job graph, -changing parallelism, forking a second job like for a red/blue deployment, and so on. Of course, Savepoints must survive job termination. Conceptually, Savepoints can be a bit more expensive to produce and restore from and focus -more on portability and flexibility with respect to changes to the job. +To make proper use of savepoints, it's important to understand the differences between [checkpoints]({{< ref "docs/ops/state/checkpoints" >}}) and savepoints which is described in [checkpoints vs. savepoints]({{< ref "docs/ops/state/checkpoints_vs_savepoints" >}}). ## Assigning Operator IDs @@ -256,8 +249,8 @@ of checkpoints. {{< hint info >}} **Attention:** -1. Retained checkpoints are stored in a path like `//chk_`. Flink does not -take ownership of the `/` directory, but only the `chk_`. The directory +1. Retained checkpoints are stored in a path like `//chk-`. Flink does not +take ownership of the `/` directory, but only the `chk-`. The directory of the old job will not be deleted by Flink 2. [Native](#savepoint-format) format supports incremental RocksDB savepoints. For those savepoints Flink puts all diff --git a/docs/content/docs/ops/state/state_backends.md b/docs/content/docs/ops/state/state_backends.md index aa06c909fb336..3465de8a35f04 100644 --- a/docs/content/docs/ops/state/state_backends.md +++ b/docs/content/docs/ops/state/state_backends.md @@ -61,6 +61,8 @@ The HashMapStateBackend is encouraged for: It is also recommended to set [managed memory]({{< ref "docs/deployment/memory/mem_setup_tm" >}}#managed-memory) to zero. This will ensure that the maximum amount of memory is allocated for user code on the JVM. +Unlike EmbeddedRocksDBStateBackend, the HashMapStateBackend stores data as objects on the heap so that it is unsafe to reuse objects. + ### The EmbeddedRocksDBStateBackend The EmbeddedRocksDBStateBackend holds in-flight data in a [RocksDB](http://rocksdb.org) database @@ -83,7 +85,7 @@ Note that the amount of state that you can keep is only limited by the amount of This allows keeping very large state, compared to the HashMapStateBackend that keeps state in memory. This also means, however, that the maximum throughput that can be achieved will be lower with this state backend. All reads/writes from/to this backend have to go through de-/serialization to retrieve/store the state objects, which is also more expensive than always working with the -on-heap representation as the heap-based backends are doing. +on-heap representation as the heap-based backends are doing. It's safe for EmbeddedRocksDBStateBackend to reuse objects due to the de-/serialization. Check also recommendations about the [task executor memory configuration]({{< ref "docs/deployment/memory/mem_tuning" >}}#rocksdb-state-backend) for the EmbeddedRocksDBStateBackend. @@ -419,7 +421,7 @@ env.enable_changelog_statebackend(true) ### Monitoring -Available metrics are listed [here]({{< ref "docs/ops/metrics#changelog" >}}). +Available metrics are listed [here]({{< ref "docs/ops/metrics#state-changelog" >}}). If a task is backpressured by writing state changes, it will be shown as busy (red) in the UI. diff --git a/docs/content/docs/ops/upgrading.md b/docs/content/docs/ops/upgrading.md index 8b1480d8e9181..fb3fd159416cf 100644 --- a/docs/content/docs/ops/upgrading.md +++ b/docs/content/docs/ops/upgrading.md @@ -231,6 +231,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: 1.11.x 1.12.x 1.13.x + 1.14.x + 1.15.x Limitations @@ -250,6 +252,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: + + The maximum parallelism of a job that was migrated from Flink 1.1.x to 1.2.x+ is currently fixed as the parallelism of the job. This means that the parallelism can not be increased after migration. This limitation might be removed in a future bugfix release. @@ -269,6 +273,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O When migrating from Flink 1.2.x to Flink 1.3.x+, changing parallelism at the same time is not supported. Users have to first take a savepoint after migrating to Flink 1.3.x+, and then change @@ -294,6 +300,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O Migrating from Flink 1.3.0 to Flink 1.4.[0,1] will fail if the savepoint contains Scala case classes. Users have to directly migrate to 1.4.2+ instead. @@ -311,6 +319,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -328,6 +338,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O There is a known issue with resuming broadcast state created with 1.5.x in versions 1.6.x up to 1.6.2, and 1.7.0: FLINK-11087. Users upgrading to 1.6.x or 1.7.x series need to directly migrate to minor versions higher than 1.6.2 and 1.7.0, @@ -348,6 +360,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -365,6 +379,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -382,6 +398,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -399,6 +417,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -416,6 +436,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -433,6 +455,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -450,6 +474,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O + O + O @@ -467,8 +493,48 @@ Savepoints are compatible across Flink versions as indicated by the table below: O + O + O Don't upgrade from 1.12.x to 1.13.x with an unaligned checkpoint. Please use a savepoint for migrating. + + 1.14.x + + + + + + + + + + + + + + O + O + + + + 1.15.x + + + + + + + + + + + + + + + O + + diff --git a/docs/data/sql_functions.yml b/docs/data/sql_functions.yml index 822815fc622b7..8c01be4d5753c 100644 --- a/docs/data/sql_functions.yml +++ b/docs/data/sql_functions.yml @@ -221,10 +221,10 @@ arithmetic: description: Returns a pseudorandom double value in the range [0.0, 1.0) with an initial seed integer. Two RAND functions will return identical sequences of numbers if they have the same initial seed. - sql: RAND_INTEGER(INT) table: randInteger(INT) - description: Returns a pseudorandom double value in the range [0.0, INT) + description: Returns a pseudorandom integer value in the range [0, INT) - sql: RAND_INTEGER(INT1, INT2) table: randInteger(INT1, INT2) - description: Returns a pseudorandom double value in the range [0.0, INT2) with an initial seed INT1. Two RAND_INTGER functions will return idential sequences of numbers if they have the same initial seed and bound. + description: Returns a pseudorandom integer value in the range [0, INT2) with an initial seed INT1. Two RAND_INTGER functions will return idential sequences of numbers if they have the same initial seed and bound. - sql: UUID() table: uuid() description: Returns an UUID (Universally Unique Identifier) string (e.g., "3d3c68f7-f608-473f-b60c-b0c44ad4cc4e") according to RFC 4122 type 4 (pseudo randomly generated) UUID. The UUID is generated using a cryptographically strong pseudo random number generator. @@ -380,8 +380,7 @@ temporal: An interval range might be DAY, MINUTE, DAY TO HOUR, or DAY TO SECOND for intervals of milliseconds; YEAR or YEAR TO MONTH for intervals of months. E.g., INTERVAL '10 00:00:00.004' DAY TO SECOND, INTERVAL '10' DAY, or INTERVAL '2-10' YEAR TO MONTH return intervals. - - sql: YEAR(date) - table: | + - table: | NUMERIC.year NUMERIC.years description: Creates an interval of months for NUMERIC years. diff --git a/docs/data/sql_functions_zh.yml b/docs/data/sql_functions_zh.yml index 4d795ab47a7e2..8fc1e4d663d27 100644 --- a/docs/data/sql_functions_zh.yml +++ b/docs/data/sql_functions_zh.yml @@ -271,11 +271,11 @@ arithmetic: 如果两个 RAND 函数具有相同的初始种子,它们将返回相同的数字序列。 - sql: RAND_INTEGER(INT) table: randInteger(INT) - description: 返回 [0.0, INT) 范围内的伪随机双精度值。 + description: 返回 [0, INT) 范围内的伪随机整数。 - sql: RAND_INTEGER(INT1, INT2) table: randInteger(INT1, INT2) description: | - 返回范围为 [0.0, INT2) 的伪随机双精度值,初始种子为 INT1。 + 返回范围为 [0, INT2) 的伪随机整数,初始种子为 INT1。 如果两个 RAND_INTGER 函数具有相同的初始种子和边界,它们将返回相同的数字序列。 - sql: UUID() table: uuid() @@ -480,8 +480,7 @@ temporal: 是 DAY,MINUTE,DAY TO HOUR 或 DAY TO SECOND,以毫秒为间隔;YEAR 或 YEAR TO MONTH 表示几个月的间隔。例 如 `INTERVAL '10 00:00:00.004' DAY TO SECOND,INTERVAL '10' DAY` 或 `INTERVAL '2-10' YEAR TO MONTH` 返回间隔。 - - sql: YEAR(date) - table: numeric.year | numeric.years + - table: numeric.year | numeric.years description: 创建 numeric 年的月间隔。 - table: numeric.quarter | numeric.quarters description: 为 numeric 季度创建月间隔。例如 `2.quarters` 返回 `6`。 @@ -708,11 +707,10 @@ json: - sql: IS JSON [ { VALUE | SCALAR | ARRAY | OBJECT } ] table: STRING.isJson([JsonType type]) description: | - Determine whether a given string is valid JSON. + 判定给定字符串是否为有效的 JSON。 - Specifying the optional type argument puts a constraint on which type of JSON object is - allowed. If the string is valid JSON, but not that type, `false` is returned. The default is - `VALUE`. + 指定可选类型参数将会限制 JSON 对象所允许的类型。 + 如果字符串是有效的 JSON,但不是指定的类型,则返回 `false`。默认值为 `VALUE`。 ``` -- TRUE @@ -743,9 +741,9 @@ json: - sql: JSON_EXISTS(jsonValue, path [ { TRUE | FALSE | UNKNOWN | ERROR } ON ERROR ]) table: STRING.jsonExists(STRING path [, JsonExistsOnError onError]) description: | - Determines whether a JSON string satisfies a given path search criterion. + 判定 JSON 字符串是否满足给定的路径搜索条件。 - If the error behavior is omitted, `FALSE ON ERROR` is assumed as the default. + 如果要忽略错误行为,那么将 `FALSE ON ERROR` 设为默认值。 ``` -- TRUE @@ -766,10 +764,9 @@ json: - sql: JSON_STRING(value) table: jsonString(value) description: | - Serializes a value into JSON. + 将值序列化为 JSON。 - This function returns a JSON string containing the serialized value. If the value is `NULL`, - the function returns `NULL`. + 此函数返回一个包含序列化值的 JSON 字符串。如果值为 `NULL`,函数返回 `NULL`。 ``` -- NULL @@ -787,23 +784,18 @@ json: - sql: JSON_VALUE(jsonValue, path [RETURNING ] [ { NULL | ERROR | DEFAULT } ON EMPTY ] [ { NULL | ERROR | DEFAULT } ON ERROR ]) table: STRING.jsonValue(STRING path [, returnType, onEmpty, defaultOnEmpty, onError, defaultOnError]) description: | - Extracts a scalar from a JSON string. + 从 JSON 字符串中提取标量。 - This method searches a JSON string for a given path expression and returns the value if the - value at that path is scalar. Non-scalar values cannot be returned. By default, the value is - returned as `STRING`. Using `returningType` a different type can be chosen, with the following - types being supported: + 此方法搜索给定路径表达式的 JSON 字符串,如果该路径上的值是标量则返回该值。不能返回非标量值。默认情况下返回值类型为 `STRING`。 + 可以使用 `returnsType` 设置不同的类型,如下所示: * `VARCHAR` / `STRING` * `BOOLEAN` * `INTEGER` * `DOUBLE` - For empty path expressions or errors a behavior can be defined to either return `null`, raise - an error or return a defined default value instead. When omitted, the default is - `NULL ON EMPTY` or `NULL ON ERROR`, respectively. The default value may be a literal or an - expression. If the default value itself raises an error, it falls through to the error - behavior for `ON EMPTY`, and raises an error for `ON ERROR`. + 对于空路径表达式或错误,可以将行为定义为返回 `null`、引发错误或返回定义的默认值。当省略时,默认为 `NULL ON EMPTY` 或 + `NULL ON ERROR`。默认值可以是文字或表达式。如果默认值本身引发错误,通过错误就会造成 `ON EMPTY` 的行为,并引发 `ON ERROR` 的错误。 ``` -- "true" @@ -821,22 +813,20 @@ json: DEFAULT FALSE ON ERROR) -- 0.998D - JSON_VALUE('{"a.b": [0.998,0.996]}','$.["a.b"][0]' + JSON_VALUE('{"a.b": [0.998,0.996]}','$.["a.b"][0]' RETURNING DOUBLE) ``` - sql: JSON_QUERY(jsonValue, path [ { WITHOUT | WITH CONDITIONAL | WITH UNCONDITIONAL } [ ARRAY ] WRAPPER ] [ { NULL | EMPTY ARRAY | EMPTY OBJECT | ERROR } ON EMPTY ] [ { NULL | EMPTY ARRAY | EMPTY OBJECT | ERROR } ON ERROR ]) table: STRING.jsonQuery(path [, JsonQueryWrapper [, JsonQueryOnEmptyOrError, JsonQueryOnEmptyOrError ] ]) description: | - Extracts JSON values from a JSON string. + 从 JSON 字符串中提取 JSON 值。 - The result is always returned as a `STRING`. The `RETURNING` clause is currently not supported. + 结果总是以 `STRING` 的形式返回。目前尚不支持 `RETURNING` 子句。 - The `wrappingBehavior` determines whether the extracted value should be wrapped into an array, - and whether to do so unconditionally or only if the value itself isn't an array already. + `wrappingBehavior` 决定是否将提取的值包装到一个数组中,以及是否无条件地这样做,还是只有当值本身不是数组时才这样做。 - `onEmpty` and `onError` determine the behavior in case the path expression is empty, or in - case an error was raised, respectively. By default, in both cases `null` is returned. Other - choices are to use an empty array, an empty object, or to raise an error. + `onEmpty` 和 `onError` 分别决定路径表达式为空或引发错误时的行为。默认情况下,这两种情况都会返回 `null`。其他选择是 + 使用空数组、空对象或引发错误。 ``` -- '{ "b": 1 }' @@ -849,7 +839,7 @@ json: JSON_QUERY('{"a":[{"c":"c1"},{"c":"c2"}]}', 'lax $.a[*].c') - -- Wrap result into an array + -- 将结果包装到数组中 -- '[{}]' JSON_QUERY('{}', '$' WITH CONDITIONAL ARRAY WRAPPER) -- '[1, 2]' @@ -857,13 +847,13 @@ json: -- '[[1, 2]]' JSON_QUERY('[1, 2]', '$' WITH UNCONDITIONAL ARRAY WRAPPER) - -- Scalars must be wrapped to be returned + -- 必须包装标量才能返回 -- NULL JSON_QUERY(1, '$') -- '[1]' JSON_QUERY(1, '$' WITH CONDITIONAL ARRAY WRAPPER) - -- Behavior if path expression is empty / there is an error + -- 路径表达式为空/存在错误时的行为 -- '{}' JSON_QUERY('{}', 'lax $.invalid' EMPTY OBJECT ON EMPTY) -- '[]' @@ -872,16 +862,13 @@ json: - sql: JSON_OBJECT([[KEY] key VALUE value]* [ { NULL | ABSENT } ON NULL ]) table: jsonObject(JsonOnNull, keyValues...) description: | - Builds a JSON object string from a list of key-value pairs. + 将键值对列表构建为 JSON 对象字符串。 - Note that keys must be non-`NULL` string literals, while values may be arbitrary expressions. + 注意,键必须是非 `NULL` 字符串自变量,而值可以是任意表达式。 - This function returns a JSON string. The `ON NULL` behavior defines how to treat `NULL` - values. If omitted, `NULL ON NULL` is assumed by default. + 这个函数返回一个 JSON 字符串。`ON NULL` 行为定义了如何处理 `NULL` 值。如果省略,则默认为 `NULL ON NULL`。 - Values which are created from another JSON construction function call (`JSON_OBJECT`, - `JSON_ARRAY`) are inserted directly rather than as a string. This allows building nested JSON - structures. + 值是由另一个 JSON 构造函数调用 (`JSON_OBJECT`,`JSON_ARRAY`) 直接插入所创建,而不是作为一个字符串。它允许构建嵌套的 JSON 结构。 ``` -- '{}' @@ -890,7 +877,7 @@ json: -- '{"K1":"V1","K2":"V2"}' JSON_OBJECT('K1' VALUE 'V1', 'K2' VALUE 'V2') - -- Expressions as values + -- 表达式作为值 JSON_OBJECT('orderNo' VALUE orders.orderId) -- ON NULL @@ -909,15 +896,14 @@ json: - sql: JSON_OBJECTAGG([KEY] key VALUE value [ { NULL | ABSENT } ON NULL ]) table: jsonObjectAgg(JsonOnNull, keyExpression, valueExpression) description: | - Builds a JSON object string by aggregating key-value expressions into a single JSON object. + 通过将 key-value 聚合到单个 JSON 对象中,构建 JSON 对象字符串。 - The key expression must return a non-nullable character string. Value expressions can be - arbitrary, including other JSON functions. If a value is `NULL`, the `ON NULL` behavior - defines what to do. If omitted, `NULL ON NULL` is assumed by default. + 键表达式必须返回不为空的字符串。值表达式可以是任意的,包括其他 JSON 函数。 + 如果值为 `NULL`,则 `ON NULL` 行为定义了要执行的操作。如果省略,默认情况下假定为 `NULL ON NULL`。 - Note that keys must be unique. If a key occurs multiple times, an error will be thrown. + 请注意,键必须是唯一的。如果一个键出现多次,将抛出一个错误。 - This function is currently not supported in `OVER` windows. + 目前在 `OVER` windows 中不支持此函数。 ``` -- '{"Apple":2,"Banana":17,"Orange":0}' @@ -928,15 +914,11 @@ json: - sql: JSON_ARRAY([value]* [ { NULL | ABSENT } ON NULL ]) table: jsonArray(JsonOnNull, values...) description: | - Builds a JSON array string from a list of values. + 将数值列表构建为 JSON 数组字符串。 - This function returns a JSON string. The values can be arbitrary expressions. The `ON NULL` - behavior defines how to treat `NULL` values. If omitted, `ABSENT ON NULL` is assumed by - default. + 这个函数返回一个 JSON 字符串,值可以是任意表达式。`ON NULL` 行为定义了如何处理 `NULL` 值。如果省略,则假定 `ABSENT ON NULL` 为默认值。 - Elements which are created from another JSON construction function call (`JSON_OBJECT`, - `JSON_ARRAY`) are inserted directly rather than as a string. This allows building nested JSON - structures. + 元素是由另一个 JSON 构造函数调用 (`JSON_OBJECT`,`JSON_ARRAY`) 直接插入所创建,而不是作为一个字符串。它允许构建嵌套的 JSON 结构。 ``` -- '[]' @@ -944,7 +926,7 @@ json: -- '[1,"2"]' JSON_ARRAY(1, '2') - -- Expressions as values + -- 表达式作为值 JSON_ARRAY(orders.orderId) -- ON NULL @@ -957,13 +939,11 @@ json: - sql: JSON_ARRAYAGG(items [ { NULL | ABSENT } ON NULL ]) table: jsonArrayAgg(JsonOnNull, itemExpression) description: | - Builds a JSON object string by aggregating items into an array. + 通过将字段聚合到数组中构建 JSON 对象字符串。 - Item expressions can be arbitrary, including other JSON functions. If a value is `NULL`, the - `ON NULL` behavior defines what to do. If omitted, `ABSENT ON NULL` is assumed by default. + 项目表达式可以是任意的,包括其他 JSON 函数。如果值为 `NULL`,则 `ON NULL` 行为定义了要执行的操作。如果省略,默认情况下假定为 `ABSENT ON NULL`。 - This function is currently not supported in `OVER` windows, unbounded session windows, or hop - windows. + 此函数目前不支持 `OVER` windows、未绑定的 session windows 或 hop windows。 ``` -- '["Apple","Banana","Orange"]' diff --git a/docs/layouts/shortcodes/generated/cleanup_configuration.html b/docs/layouts/shortcodes/generated/cleanup_configuration.html index 02e861816713d..e0663c3648c1f 100644 --- a/docs/layouts/shortcodes/generated/cleanup_configuration.html +++ b/docs/layouts/shortcodes/generated/cleanup_configuration.html @@ -12,7 +12,7 @@
cleanup-strategy
"exponential-delay" String - Defines the cleanup strategy to use in case of cleanup failures.
Accepted values are:
  • none, disable, off: Cleanup is only performed once. No retry will be initiated in case of failure.
  • fixed-delay, fixeddelay: Cleanup attempts will be separated by a fixed interval up to the point where the cleanup is considered successful or a set amount of retries is reached.
  • exponential-delay, exponentialdelay: Exponential delay restart strategy triggers the cleanup with an exponentially increasing delay up to the point where the cleanup succeeded or a set amount of retries is reached.
The default configuration relies on an exponentially delayed retry strategy with the given default values. + Defines the cleanup strategy to use in case of cleanup failures.
Accepted values are:
  • none, disable, off: Cleanup is only performed once. No retry will be initiated in case of failure. The job artifacts (and the job's JobResultStore entry) have to be cleaned up manually in case of a failure.
  • fixed-delay, fixeddelay: Cleanup attempts will be separated by a fixed interval up to the point where the cleanup is considered successful or a set amount of retries is reached. Reaching the configured limit means that the job artifacts (and the job's JobResultStore entry) might need to be cleaned up manually.
  • exponential-delay, exponentialdelay: Exponential delay restart strategy triggers the cleanup with an exponentially increasing delay up to the point where the cleanup succeeded or a set amount of retries is reached. Reaching the configured limit means that the job artifacts (and the job's JobResultStore entry) might need to be cleaned up manually.
The default configuration relies on an exponentially delayed retry strategy with the given default values. diff --git a/docs/layouts/shortcodes/generated/execution_config_configuration.html b/docs/layouts/shortcodes/generated/execution_config_configuration.html index 74cf7f2e3daa5..364d9bf9d814c 100644 --- a/docs/layouts/shortcodes/generated/execution_config_configuration.html +++ b/docs/layouts/shortcodes/generated/execution_config_configuration.html @@ -42,7 +42,7 @@
table.exec.legacy-cast-behaviour

Batch Streaming - ENABLED + DISABLED

Enum

Determines whether CAST will operate following the legacy behaviour or the new one that introduces various fixes and improvements.

Possible values:
  • "ENABLED": CAST will operate following the legacy behaviour.
  • "DISABLED": CAST will operate following the new correct behaviour.
diff --git a/docs/layouts/shortcodes/generated/exponential_delay_cleanup_strategy_configuration.html b/docs/layouts/shortcodes/generated/exponential_delay_cleanup_strategy_configuration.html index f301253418b1e..23f13e0a5145c 100644 --- a/docs/layouts/shortcodes/generated/exponential_delay_cleanup_strategy_configuration.html +++ b/docs/layouts/shortcodes/generated/exponential_delay_cleanup_strategy_configuration.html @@ -10,9 +10,9 @@
cleanup-strategy.exponential-delay.attempts
- (none) + infinite Integer - The number of times a failed cleanup is retried if cleanup-strategy has been set to exponential-delay. (no value means: infinitely). + The number of times a failed cleanup is retried if cleanup-strategy has been set to exponential-delay. Reaching the configured limit means that the job artifacts (and the job's JobResultStore entry) might need to be cleaned up manually.
cleanup-strategy.exponential-delay.initial-backoff
diff --git a/docs/layouts/shortcodes/generated/fixed_delay_cleanup_strategy_configuration.html b/docs/layouts/shortcodes/generated/fixed_delay_cleanup_strategy_configuration.html index 6890b1b8292b5..231fc9d07b028 100644 --- a/docs/layouts/shortcodes/generated/fixed_delay_cleanup_strategy_configuration.html +++ b/docs/layouts/shortcodes/generated/fixed_delay_cleanup_strategy_configuration.html @@ -10,13 +10,13 @@
cleanup-strategy.fixed-delay.attempts
- 1 + infinite Integer - The number of times that Flink retries the cleanup before giving up if cleanup-strategy has been set to fixed-delay. + The number of times that Flink retries the cleanup before giving up if cleanup-strategy has been set to fixed-delay. Reaching the configured limit means that the job artifacts (and the job's JobResultStore entry) might need to be cleaned up manually.
cleanup-strategy.fixed-delay.delay
- 1 s + 1 min Duration Amount of time that Flink waits before re-triggering the cleanup after a failed attempt if the cleanup-strategy is set to fixed-delay. It can be specified using the following notation: "1 min", "20 s" diff --git a/docs/layouts/shortcodes/generated/pulsar_catalog_factory_configuration.html b/docs/layouts/shortcodes/generated/pulsar_catalog_factory_configuration.html new file mode 100644 index 0000000000000..3954b7735ea13 --- /dev/null +++ b/docs/layouts/shortcodes/generated/pulsar_catalog_factory_configuration.html @@ -0,0 +1,48 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
catalog-admin-url
"http://localhost:8080"String(Required) The admin URL of the Pulsar cluster.
catalog-auth-params
(none)StringThe authentication parameters for accessing the Pulsar cluster.
catalog-auth-plugin
(none)StringThe name of the authentication plugin for accessing the Pulsar cluster.
catalog-service-url
"pulsar://localhost:6650"String(Required) The service URL of the Pulsar cluster.
catalog-tenant
"__flink_catalog"StringThe Pulsar tenant that stores all table information.
default-database
"default_database"StringThe default database when using PulsarCatalog. It will be created if it does not exist.
diff --git a/docs/layouts/shortcodes/generated/pulsar_client_configuration.html b/docs/layouts/shortcodes/generated/pulsar_client_configuration.html index 02a6d96867e1a..970b847ac2ab9 100644 --- a/docs/layouts/shortcodes/generated/pulsar_client_configuration.html +++ b/docs/layouts/shortcodes/generated/pulsar_client_configuration.html @@ -100,7 +100,7 @@
pulsar.client.memoryLimitBytes
- 0 + 67108864 Long The limit (in bytes) on the amount of direct memory that will be allocated by this client instance.
Note: at this moment this is only limiting the memory for producers. Setting this to 0 will disable the limit. diff --git a/docs/layouts/shortcodes/generated/pulsar_consumer_configuration.html b/docs/layouts/shortcodes/generated/pulsar_consumer_configuration.html index bc8b6df40608b..4e05b270de4b0 100644 --- a/docs/layouts/shortcodes/generated/pulsar_consumer_configuration.html +++ b/docs/layouts/shortcodes/generated/pulsar_consumer_configuration.html @@ -140,12 +140,6 @@ Boolean If enabled, the consumer will automatically retry messages. - -
pulsar.consumer.subscriptionInitialPosition
- Latest -

Enum

- Initial position at which to set cursor when subscribing to a topic at first time.

Possible values:
  • "Latest"
  • "Earliest"
-
pulsar.consumer.subscriptionMode
Durable diff --git a/docs/layouts/shortcodes/generated/pulsar_producer_configuration.html b/docs/layouts/shortcodes/generated/pulsar_producer_configuration.html new file mode 100644 index 0000000000000..94739029031a6 --- /dev/null +++ b/docs/layouts/shortcodes/generated/pulsar_producer_configuration.html @@ -0,0 +1,90 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
pulsar.producer.batchingEnabled
trueBooleanEnable batch send ability, it was enabled by default.
pulsar.producer.batchingMaxBytes
131072IntegerThe maximum size of messages permitted in a batch. Keep the maximum consistent as previous versions.
pulsar.producer.batchingMaxMessages
1000IntegerThe maximum number of messages permitted in a batch.
pulsar.producer.batchingMaxPublishDelayMicros
1000LongBatching time period of sending messages.
pulsar.producer.batchingPartitionSwitchFrequencyByPublishDelay
10IntegerThe maximum wait time for switching topic partitions.
pulsar.producer.chunkingEnabled
falseBoolean
pulsar.producer.compressionType
NONE

Enum

Message data compression type used by a producer.Available options:

Possible values:
  • "NONE"
  • "LZ4"
  • "ZLIB"
  • "ZSTD"
  • "SNAPPY"
pulsar.producer.encryptionKeys
(none)List<String>Add public encryption key, used by producer to encrypt the data key.
pulsar.producer.initialSequenceId
(none)LongThe sequence id for avoiding the duplication, it's used when Pulsar doesn't have transaction.
pulsar.producer.producerCryptoFailureAction
FAIL

Enum

The action the producer will take in case of encryption failures.

Possible values:
  • "FAIL"
  • "SEND"
pulsar.producer.producerName
(none)StringA producer name which would be displayed in the Pulsar's dashboard. If no producer name was provided, we would use a Pulsar generated name instead.
pulsar.producer.properties
MapA name or value property of this consumer. properties is application defined metadata attached to a consumer. When getting a topic stats, associate this metadata with the consumer stats for easier identification.
pulsar.producer.sendTimeoutMs
30000LongMessage send timeout in ms.If a message is not acknowledged by a server before the sendTimeout expires, an error occurs.
diff --git a/docs/layouts/shortcodes/generated/pulsar_sink_configuration.html b/docs/layouts/shortcodes/generated/pulsar_sink_configuration.html new file mode 100644 index 0000000000000..cd7f803534a1e --- /dev/null +++ b/docs/layouts/shortcodes/generated/pulsar_sink_configuration.html @@ -0,0 +1,48 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
pulsar.sink.deliveryGuarantee
none

Enum

Optional delivery guarantee when committing.

Possible values:
  • "exactly-once": Records are only delivered exactly-once also under failover scenarios. To build a complete exactly-once pipeline is required that the source and sink support exactly-once and are properly configured.
  • "at-least-once": Records are ensured to be delivered but it may happen that the same record is delivered multiple times. Usually, this guarantee is faster than the exactly-once delivery.
  • "none": Records are delivered on a best effort basis. It is often the fastest way to process records but it may happen that records are lost or duplicated.
pulsar.sink.enableSchemaEvolution
falseBooleanIf you enable this option and use PulsarSerializationSchema.pulsarSchema(), we would consume and deserialize the message by using Pulsar's Schema.
pulsar.sink.maxRecommitTimes
5IntegerThe allowed transaction recommit times if we meet some retryable exception. This is used in Pulsar Transaction.
pulsar.sink.messageKeyHash
murmur-3-32-hash

Enum

The hash policy for routing message by calculating the hash code of message key.

Possible values:
  • "java-hash": This hash would use String.hashCode() to calculate the message key string's hash code.
  • "murmur-3-32-hash": This hash would calculate message key's hash code by using Murmur3 algorithm.
pulsar.sink.topicMetadataRefreshInterval
1800000LongAuto update the topic metadata in a fixed interval (in ms). The default value is 30 minutes.
pulsar.sink.transactionTimeoutMillis
10800000LongThis option is used when the user require the DeliveryGuarantee.EXACTLY_ONCE semantic.We would use transaction for making sure the message could be write only once.
diff --git a/docs/layouts/shortcodes/generated/pulsar_source_configuration.html b/docs/layouts/shortcodes/generated/pulsar_source_configuration.html index 3bcdad85e1a04..5f25117a45a9a 100644 --- a/docs/layouts/shortcodes/generated/pulsar_source_configuration.html +++ b/docs/layouts/shortcodes/generated/pulsar_source_configuration.html @@ -14,12 +14,24 @@ Long This option is used only when the user disables the checkpoint and uses Exclusive or Failover subscription. We would automatically commit the cursor using the given period (in ms). + +
pulsar.source.defaultFetchTime
+ 100 + Long + The time (in ms) for fetching messages from Pulsar. If time exceed and no message returned from Pulsar. We would consider there is no record at the current topic and stop fetch until next switch. +
pulsar.source.enableAutoAcknowledgeMessage
false Boolean Flink commits the consuming position with pulsar transactions on checkpoint. However, if you have disabled the Flink checkpoint or disabled transaction for your Pulsar cluster, ensure that you have set this option to true.
The source would use pulsar client's internal mechanism and commit cursor in two ways.
  • For Key_Shared and Shared subscription, the cursor would be committed once the message is consumed.
  • For Exclusive and Failover subscription, the cursor would be committed in a given interval.
+ +
pulsar.source.enableSchemaEvolution
+ false + Boolean + If you enable this option, we would consume and deserialize the message by using Pulsar's Schema. +
pulsar.source.maxFetchRecords
100 diff --git a/docs/layouts/shortcodes/generated/pulsar_table_configuration.html b/docs/layouts/shortcodes/generated/pulsar_table_configuration.html new file mode 100644 index 0000000000000..f87fa1766d416 --- /dev/null +++ b/docs/layouts/shortcodes/generated/pulsar_table_configuration.html @@ -0,0 +1,114 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
admin-url
(none)StringThe Pulsar service HTTP URL for the admin endpoint. For example, http://my-broker.example.com:8080, or https://my-broker.example.com:8443 for TLS.
explicit
trueBooleanIndicate if the table is an explicit Flink table.
key.fields
List<String>An explicit list of physical columns from the table schema that are decoded/encoded from the key bytes of a Pulsar message. By default, this list is empty and thus a key is undefined.
key.format
(none)StringDefines the format identifier for decoding/encoding key bytes in Pulsar message. The identifier is used to discover a suitable format factory.
service-url
(none)StringService URL provider for Pulsar service.
To connect to Pulsar using client libraries, you need to specify a Pulsar protocol URL.
You can assign Pulsar protocol URLs to specific clusters and use the Pulsar scheme.
  • This is an example of localhost: pulsar://localhost:6650.
  • If you have multiple brokers, the URL is as: pulsar://localhost:6550,localhost:6651,localhost:6652
  • A URL for a production Pulsar cluster is as: pulsar://pulsar.us-west.example.com:6650
  • If you use TLS authentication, the URL is as pulsar+ssl://pulsar.us-west.example.com:6651
sink.custom-topic-router
(none)String(Optional) the custom topic router class URL that is used in the [Pulsar DataStream sink connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-sink). If this option is provided, the sink.topic-routing-mode option will be ignored.
sink.message-delay-interval
0 msDuration(Optional) the message delay delivery interval that is used in the [Pulsar DataStream sink connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-sink).
sink.topic-routing-mode
round-robin

Enum

(Optional) the topic routing mode. Available options are round-robin and message-key-hash. By default, it is set to round-robin. If you want to use a custom topic router, use the sink.custom-topic-router option to determine the partition for a particular message.

Possible values:
  • "round-robin": The producer will publish messages across all partitions in a round-robin fashion to achieve maximum throughput. Please note that round-robin is not done per individual message but rather it's set to the same boundary of pulsar.producer.batchingMaxMessages, to ensure batching is effective.
  • "message-key-hash": If no key is provided, The partitioned producer will randomly pick one single topic partition and publish all the messages into that partition. If a key is provided on the message, the partitioned producer will hash the key and assign the message to a particular partition.
  • "custom": Use custom TopicRouter implementation that will be called to determine the partition for a particular message.
source.start.message-id
(none)StringOptional message id used to specify a consuming starting point for source. Use earliest, latest or pass in a message id representation in ledgerId:entryId:partitionId, such as 12:2:-1
source.start.publish-time
(none)Long(Optional) the publish timestamp that is used to specify a starting point for the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source) to consume data.
source.stop.after-message-id
(none)StringOptional message id used to specify a stop position but include the given message in the consuming result for the unbounded sql source. Pass in a message id representation in "ledgerId:entryId:partitionId", such as "12:2:-1".
source.stop.at-message-id
(none)StringOptional message id used to specify a stop cursor for the unbounded sql source. Use "never", "latest" or pass in a message id representation in "ledgerId:entryId:partitionId", such as "12:2:-1"
source.stop.at-publish-time
(none)LongOptional publish timestamp used to specify a stop cursor for the unbounded sql source.
source.subscription-name
(none)StringThe subscription name of the consumer that is used by the runtime [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source). This argument is required for constructing the consumer.
source.subscription-type
Exclusive

Enum

The [subscription type](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-subscriptions) that is supported by the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source). Currently, only Exclusive and Shared subscription types are supported.

Possible values:
  • "Exclusive"
  • "Shared"
  • "Failover"
  • "Key_Shared"
topics
(none)List<String>Topic name(s) the table reads data from. It can be a single topic name or a list of topic names separated by a semicolon symbol (;) like topic-1;topic-2.
value.format
(none)StringDefines the format identifier for decoding/encoding value data. The identifier is used to discover a suitable format factory.
diff --git a/docs/layouts/shortcodes/generated/python_configuration.html b/docs/layouts/shortcodes/generated/python_configuration.html index 9ad42d8c580d0..cdaabcf4621b7 100644 --- a/docs/layouts/shortcodes/generated/python_configuration.html +++ b/docs/layouts/shortcodes/generated/python_configuration.html @@ -30,7 +30,7 @@
python.execution-mode
"process" String - Specify the python runtime execution mode. The optional values are `process`, `multi-thread` and `sub-interpreter`. The `process` mode means that the Python user-defined functions will be executed in separate Python process. The `multi-thread` mode means that the Python user-defined functions will be executed in the same thread as Java Operator, but it will be affected by GIL performance. The `sub-interpreter` mode means that the Python user-defined functions will be executed in python different sub-interpreters rather than different threads of one interpreter, which can largely overcome the effects of the GIL, but it maybe fail in some CPython extensions libraries, such as numpy, tensorflow. Note that if the python operator dose not support `multi-thread` and `sub-interpreter` mode, we will still use `process` mode. + Specify the python runtime execution mode. The optional values are `process` and `thread`. The `process` mode means that the Python user-defined functions will be executed in separate Python process. The `thread` mode means that the Python user-defined functions will be executed in the same process of the Java operator. Note that currently it still doesn't support to execute Python user-defined functions in `thread` mode in all places. It will fall back to `process` mode in these cases.
python.files
diff --git a/docs/layouts/shortcodes/generated/state_backend_changelog_section.html b/docs/layouts/shortcodes/generated/state_backend_changelog_section.html index 405133dee789a..a5b41d308c349 100644 --- a/docs/layouts/shortcodes/generated/state_backend_changelog_section.html +++ b/docs/layouts/shortcodes/generated/state_backend_changelog_section.html @@ -24,7 +24,7 @@
state.backend.changelog.periodic-materialize.interval
10 min Duration - Defines the interval in milliseconds to perform periodic materialization for state backend. + Defines the interval in milliseconds to perform periodic materialization for state backend. The periodic materialization will be disabled when the value is negative
state.backend.changelog.storage
diff --git a/docs/layouts/shortcodes/generated/state_changelog_configuration.html b/docs/layouts/shortcodes/generated/state_changelog_configuration.html index 405133dee789a..a5b41d308c349 100644 --- a/docs/layouts/shortcodes/generated/state_changelog_configuration.html +++ b/docs/layouts/shortcodes/generated/state_changelog_configuration.html @@ -24,7 +24,7 @@
state.backend.changelog.periodic-materialize.interval
10 min Duration - Defines the interval in milliseconds to perform periodic materialization for state backend. + Defines the interval in milliseconds to perform periodic materialization for state backend. The periodic materialization will be disabled when the value is negative
state.backend.changelog.storage
diff --git a/docs/layouts/shortcodes/generated/table_config_configuration.html b/docs/layouts/shortcodes/generated/table_config_configuration.html index 53e5dbcfc2bc9..fe9784a220722 100644 --- a/docs/layouts/shortcodes/generated/table_config_configuration.html +++ b/docs/layouts/shortcodes/generated/table_config_configuration.html @@ -8,6 +8,18 @@ + +
table.builtin-catalog-name

Batch Streaming + "default_catalog" + String + The name of the initial catalog to be created when instantiating a TableEnvironment. + + +
table.builtin-database-name

Batch Streaming + "default_database" + String + The name of the default database in the initial catalog to be created when instantiating TableEnvironment. +
table.dml-sync

Batch Streaming false diff --git a/docs/layouts/shortcodes/ref_static.html b/docs/layouts/shortcodes/ref_static.html index 797d25c2f1e58..82c726939c659 100644 --- a/docs/layouts/shortcodes/ref_static.html +++ b/docs/layouts/shortcodes/ref_static.html @@ -1,3 +1,4 @@ + {{/* Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file @@ -18,4 +19,4 @@ */}}{{/* Shortcode for embedding a link to a static resource. */}} -{{ .Site.BaseURL }}{{ .Get 0 }} +{{ .Site.BaseURL }}/{{ .Get 0 }} diff --git a/docs/static/fig/hive_parser_load_exception.png b/docs/static/fig/hive_parser_load_exception.png new file mode 100644 index 0000000000000..4d7fc3cf71ea1 Binary files /dev/null and b/docs/static/fig/hive_parser_load_exception.png differ diff --git a/flink-annotations/pom.xml b/flink-annotations/pom.xml index a2260ba99e534..b07f8069c5a28 100644 --- a/flink-annotations/pom.xml +++ b/flink-annotations/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-architecture-tests/flink-architecture-tests-base/pom.xml b/flink-architecture-tests/flink-architecture-tests-base/pom.xml index 53f61c8ec85d5..69aa346256e65 100644 --- a/flink-architecture-tests/flink-architecture-tests-base/pom.xml +++ b/flink-architecture-tests/flink-architecture-tests-base/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-architecture-tests - 1.15-SNAPSHOT + 1.15.0 ../pom.xml diff --git a/flink-architecture-tests/flink-architecture-tests-production/pom.xml b/flink-architecture-tests/flink-architecture-tests-production/pom.xml index 4752e9b6f872e..5b5964bbc8461 100644 --- a/flink-architecture-tests/flink-architecture-tests-production/pom.xml +++ b/flink-architecture-tests/flink-architecture-tests-production/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-architecture-tests - 1.15-SNAPSHOT + 1.15.0 ../pom.xml diff --git a/flink-architecture-tests/flink-architecture-tests-test/pom.xml b/flink-architecture-tests/flink-architecture-tests-test/pom.xml index a5e41eff5fe5b..ebc4a54363e5c 100644 --- a/flink-architecture-tests/flink-architecture-tests-test/pom.xml +++ b/flink-architecture-tests/flink-architecture-tests-test/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-architecture-tests - 1.15-SNAPSHOT + 1.15.0 ../pom.xml diff --git a/flink-architecture-tests/pom.xml b/flink-architecture-tests/pom.xml index 788b158493cfb..8218024729665 100644 --- a/flink-architecture-tests/pom.xml +++ b/flink-architecture-tests/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 ../pom.xml diff --git a/flink-clients/pom.xml b/flink-clients/pom.xml index 72844be81429e..2410a6d0daece 100644 --- a/flink-clients/pom.xml +++ b/flink-clients/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java index 8cc4ad4c61b0c..05ac0757c1b9c 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java @@ -149,7 +149,7 @@ public class CliFrontendParser { static final Option SAVEPOINT_FORMAT_OPTION = new Option( - "t", + "type", "type", true, "Describes the binary format in which a savepoint should be taken. Supported" diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java index da4590efc2d55..2b4c6a79a1179 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java @@ -184,7 +184,7 @@ public void testTriggerSavepointCustomTarget() throws Exception { @Test public void testTriggerSavepointCustomFormatShortOption() throws Exception { - testTriggerSavepointCustomFormat("-t", SavepointFormatType.NATIVE); + testTriggerSavepointCustomFormat("-type", SavepointFormatType.NATIVE); } @Test diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopWithSavepointTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopWithSavepointTest.java index da4d40727d161..89716ca19f983 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopWithSavepointTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopWithSavepointTest.java @@ -121,7 +121,7 @@ public void testStopWithExplicitSavepointDir() throws Exception { @Test public void testStopWithExplicitSavepointTypeShortOption() throws Exception { - testStopWithExplicitSavepointType("-t", SavepointFormatType.NATIVE); + testStopWithExplicitSavepointType("-type", SavepointFormatType.NATIVE); } @Test diff --git a/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java b/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java index c20a959db4894..47036fcf3528e 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java +++ b/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.client.cli.ClientOptions; import org.apache.flink.client.deployment.application.executors.EmbeddedExecutor; import org.apache.flink.client.program.PackagedProgram; @@ -74,8 +73,6 @@ @ExtendWith(TestLoggerExtension.class) public class ApplicationDispatcherBootstrapITCase { - private static final Duration TIMEOUT = Duration.ofMinutes(10); - private static Supplier createApplicationModeDispatcherResourceManagerComponentFactorySupplier( Configuration configuration, PackagedProgram program) { @@ -97,7 +94,6 @@ public class ApplicationDispatcherBootstrapITCase { @Test public void testDispatcherRecoversAfterLosingAndRegainingLeadership() throws Exception { final String blockId = UUID.randomUUID().toString(); - final Deadline deadline = Deadline.fromNow(TIMEOUT); final Configuration configuration = new Configuration(); configuration.set(HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name()); configuration.set(DeploymentOptions.TARGET, EmbeddedExecutor.NAME); @@ -121,11 +117,7 @@ public void testDispatcherRecoversAfterLosingAndRegainingLeadership() throws Exc cluster.start(); // wait until job is running - awaitJobStatus( - cluster, - ApplicationDispatcherBootstrap.ZERO_JOB_ID, - JobStatus.RUNNING, - deadline); + awaitJobStatus(cluster, ApplicationDispatcherBootstrap.ZERO_JOB_ID, JobStatus.RUNNING); // make sure the operator is actually running BlockingJob.awaitRunning(blockId); @@ -140,11 +132,7 @@ public void testDispatcherRecoversAfterLosingAndRegainingLeadership() throws Exc haServices.grantDispatcherLeadership(); // job is suspended, wait until it's running - awaitJobStatus( - cluster, - ApplicationDispatcherBootstrap.ZERO_JOB_ID, - JobStatus.RUNNING, - deadline); + awaitJobStatus(cluster, ApplicationDispatcherBootstrap.ZERO_JOB_ID, JobStatus.RUNNING); // unblock processing so the job can finish BlockingJob.unblock(blockId); @@ -157,7 +145,7 @@ public void testDispatcherRecoversAfterLosingAndRegainingLeadership() throws Exc .isEqualTo(ApplicationStatus.SUCCEEDED); // the cluster should shut down automatically once the application completes - awaitClusterStopped(cluster, deadline); + awaitClusterStopped(cluster); } finally { BlockingJob.cleanUp(blockId); } @@ -165,7 +153,6 @@ public void testDispatcherRecoversAfterLosingAndRegainingLeadership() throws Exc @Test public void testDirtyJobResultRecoveryInApplicationMode() throws Exception { - final Deadline deadline = Deadline.fromNow(TIMEOUT); final Configuration configuration = new Configuration(); configuration.set(HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name()); configuration.set(DeploymentOptions.TARGET, EmbeddedExecutor.NAME); @@ -203,7 +190,7 @@ public JobResultStore getJobResultStore() { cluster.start(); // the cluster should shut down automatically once the application completes - awaitClusterStopped(cluster, deadline); + awaitClusterStopped(cluster); } FlinkAssertions.assertThatChainOfCauses(ErrorHandlingSubmissionJob.getSubmissionException()) @@ -223,7 +210,6 @@ public JobResultStore getJobResultStore() { @Test public void testSubmitFailedJobOnApplicationError() throws Exception { - final Deadline deadline = Deadline.fromNow(TIMEOUT); final JobID jobId = new JobID(); final Configuration configuration = new Configuration(); configuration.set(HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name()); @@ -251,7 +237,7 @@ public void testSubmitFailedJobOnApplicationError() throws Exception { cluster.start(); // wait until the failed job has been submitted - awaitJobStatus(cluster, jobId, JobStatus.FAILED, deadline); + awaitJobStatus(cluster, jobId, JobStatus.FAILED); final ArchivedExecutionGraph graph = cluster.getArchivedExecutionGraph(jobId).get(); @@ -272,13 +258,11 @@ public void testSubmitFailedJobOnApplicationError() throws Exception { } } - private static void awaitClusterStopped(MiniCluster cluster, Deadline deadline) - throws Exception { - CommonTestUtils.waitUntilCondition(() -> !cluster.isRunning(), deadline); + private static void awaitClusterStopped(MiniCluster cluster) throws Exception { + CommonTestUtils.waitUntilCondition(() -> !cluster.isRunning()); } - private static void awaitJobStatus( - MiniCluster cluster, JobID jobId, JobStatus status, Deadline deadline) + private static void awaitJobStatus(MiniCluster cluster, JobID jobId, JobStatus status) throws Exception { CommonTestUtils.waitUntilCondition( () -> { @@ -292,7 +276,6 @@ private static void awaitJobStatus( } throw e; } - }, - deadline); + }); } } diff --git a/flink-connectors/flink-connector-aws-base/pom.xml b/flink-connectors/flink-connector-aws-base/pom.xml index 431cbaca5e6d9..c99ba15fad934 100644 --- a/flink-connectors/flink-connector-aws-base/pom.xml +++ b/flink-connectors/flink-connector-aws-base/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-aws-kinesis-firehose/pom.xml b/flink-connectors/flink-connector-aws-kinesis-firehose/pom.xml index 624b66e62838c..d6b58a02aa055 100644 --- a/flink-connectors/flink-connector-aws-kinesis-firehose/pom.xml +++ b/flink-connectors/flink-connector-aws-kinesis-firehose/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. @@ -46,6 +46,12 @@ under the License. provided + + org.apache.flink + flink-connector-base + ${project.version} + + org.apache.flink flink-connector-aws-base @@ -158,34 +164,6 @@ under the License. - - org.apache.maven.plugins - maven-shade-plugin - - - shade-flink - package - - shade - - - true - - - org.apache.flink:flink-connector-base - org.apache.flink:flink-connector-aws-base - - - - - org.apache.flink.connector.base - org.apache.flink.connector.firehose.shaded.org.apache.flink.connector.base - - - - - - diff --git a/flink-connectors/flink-connector-aws-kinesis-streams/pom.xml b/flink-connectors/flink-connector-aws-kinesis-streams/pom.xml index 5620a1b38debc..7b502535a73b0 100644 --- a/flink-connectors/flink-connector-aws-kinesis-streams/pom.xml +++ b/flink-connectors/flink-connector-aws-kinesis-streams/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. @@ -46,6 +46,12 @@ under the License. provided + + org.apache.flink + flink-connector-base + ${project.version} + + org.apache.flink flink-connector-aws-base @@ -139,34 +145,6 @@ under the License. - - org.apache.maven.plugins - maven-shade-plugin - - - shade-flink - package - - shade - - - true - - - org.apache.flink:flink-connector-base - org.apache.flink:flink-connector-aws-base - - - - - org.apache.flink.connector.base - org.apache.flink.connector.kinesis.shaded.org.apache.flink.connector.base - - - - - - diff --git a/flink-connectors/flink-connector-base/pom.xml b/flink-connectors/flink-connector-base/pom.xml index 1af4fc818f5fc..f9e3646cd19ce 100644 --- a/flink-connectors/flink-connector-base/pom.xml +++ b/flink-connectors/flink-connector-base/pom.xml @@ -25,7 +25,7 @@ org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. @@ -72,6 +72,14 @@ test + + org.apache.flink + flink-streaming-java + ${project.version} + test + test-jar + + diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriter.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriter.java index f45a3a5930256..090504ab8bc61 100644 --- a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriter.java +++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriter.java @@ -69,10 +69,10 @@ public abstract class AsyncSinkWriter this.ackTime - this.lastSendTimestamp); - this.numBytesOutCounter = this.metrics.getIOMetricGroup().getNumBytesOutCounter(); - this.numRecordsOutCounter = this.metrics.getIOMetricGroup().getNumRecordsOutCounter(); + this.numBytesSendCounter = this.metrics.getNumBytesSendCounter(); + this.numRecordsSendCounter = this.metrics.getNumRecordsSendCounter(); this.fatalExceptionCons = exception -> @@ -417,8 +417,8 @@ private List createNextAvailableBatch() { batchSizeBytes += requestEntrySize; } - numRecordsOutCounter.inc(batch.size()); - numBytesOutCounter.inc(batchSizeBytes); + numRecordsSendCounter.inc(batch.size()); + numBytesSendCounter.inc(batchSizeBytes); return batch; } diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/SourceReaderBase.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/SourceReaderBase.java index 416dbb2a297d5..18d49f3caca93 100644 --- a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/SourceReaderBase.java +++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/SourceReaderBase.java @@ -324,6 +324,9 @@ private SplitContext(String splitId, SplitStateT state) { SourceOutput getOrCreateSplitOutput(ReaderOutput mainOutput) { if (sourceOutput == null) { + // The split output should have been created when AddSplitsEvent was processed in + // SourceOperator. Here we just use this method to get the previously created + // output. sourceOutput = mainOutput.createOutputForSplit(splitId); } return sourceOutput; diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/splitreader/SplitReader.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/splitreader/SplitReader.java index 4f2ff6af6d76d..550cb953e80ea 100644 --- a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/splitreader/SplitReader.java +++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/splitreader/SplitReader.java @@ -50,6 +50,11 @@ public interface SplitReader { /** * Handle the split changes. This call should be non-blocking. * + *

For the consistency of internal state in SourceReaderBase, if an invalid split is added to + * the reader (for example splits without any records), it should be put back into {@link + * RecordsWithSplitIds} as finished splits so that SourceReaderBase could be able to clean up + * resources created for it. + * * @param splitsChanges the split changes that the SplitReader needs to handle. */ void handleSplitsChanges(SplitsChange splitsChanges); diff --git a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriterTest.java b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriterTest.java index c6e95c7923a30..47c437e01e506 100644 --- a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriterTest.java +++ b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriterTest.java @@ -21,8 +21,8 @@ import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.ArrayList; @@ -41,11 +41,8 @@ import static org.apache.flink.connector.base.sink.writer.AsyncSinkWriterTestUtils.assertThatBufferStatesAreEqual; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatExceptionOfType; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.junit.jupiter.api.Assertions.fail; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assertions.fail; /** * Unit Tests the functionality of AsyncSinkWriter without any assumptions of what a concrete @@ -57,7 +54,7 @@ public class AsyncSinkWriterTest { private TestSinkInitContext sinkInitContext; private TestSinkInitContextAnyThreadMailbox sinkInitContextAnyThreadMailbox; - @Before + @BeforeEach public void before() { res.clear(); sinkInitContext = new TestSinkInitContext(); @@ -77,17 +74,20 @@ private void performNormalWriteOfEightyRecordsToMock() public void testNumberOfRecordsIsAMultipleOfBatchSizeResultsInThatNumberOfRecordsBeingWritten() throws IOException, InterruptedException { performNormalWriteOfEightyRecordsToMock(); - assertEquals(80, res.size()); + + assertThat(res.size()).isEqualTo(80); } @Test public void testMetricsGroupHasLoggedNumberOfRecordsAndNumberOfBytesCorrectly() throws IOException, InterruptedException { performNormalWriteOfEightyRecordsToMock(); - assertEquals(80, sinkInitContext.getNumRecordsOutCounter().getCount()); - assertEquals(320, sinkInitContext.getNumBytesOutCounter().getCount()); - assertTrue(sinkInitContext.getCurrentSendTimeGauge().get().getValue() >= 0); - assertTrue(sinkInitContext.getCurrentSendTimeGauge().get().getValue() < 1000); + + assertThat(sinkInitContext.getNumRecordsOutCounter().getCount()).isEqualTo(80); + assertThat(sinkInitContext.getNumBytesOutCounter().getCount()).isEqualTo(320); + assertThat(sinkInitContext.getCurrentSendTimeGauge().get().getValue()) + .isGreaterThanOrEqualTo(0); + assertThat(sinkInitContext.getCurrentSendTimeGauge().get().getValue()).isLessThan(1000); } @Test @@ -101,8 +101,10 @@ public void checkLoggedSendTimesAreWithinBounds() throws IOException, Interrupte for (int i = 0; i < 4; i++) { sink.write(String.valueOf(i)); } - assertTrue(sinkInitContext.getCurrentSendTimeGauge().get().getValue() >= 99); - assertTrue(sinkInitContext.getCurrentSendTimeGauge().get().getValue() < 110); + + assertThat(sinkInitContext.getCurrentSendTimeGauge().get().getValue()) + .isGreaterThanOrEqualTo(99); + assertThat(sinkInitContext.getCurrentSendTimeGauge().get().getValue()).isLessThan(110); } @Test @@ -113,7 +115,8 @@ public void testThatUnwrittenRecordsInBufferArePersistedWhenSnapshotIsTaken() for (int i = 0; i < 23; i++) { sink.write(String.valueOf(i)); } - assertEquals(20, res.size()); + + assertThat(res.size()).isEqualTo(20); assertThatBufferStatesAreEqual(sink.wrapRequests(20, 21, 22), getWriterState(sink)); } @@ -129,7 +132,8 @@ public void sinkToAllowBatchSizesEqualToByteWiseLimit() sink.write("1"); // 4 bytes per record sink.write("2"); // to give 12 bytes in final flush sink.write("3"); - assertEquals(3, res.size()); + + assertThat(res.size()).isEqualTo(3); } @Test @@ -141,7 +145,8 @@ public void testPreparingCommitAtSnapshotTimeEnsuresBufferedRecordsArePersistedT sink.write(String.valueOf(i)); } sink.flush(true); - assertEquals(23, res.size()); + + assertThat(res.size()).isEqualTo(23); } @Test @@ -151,7 +156,8 @@ public void testThatMailboxYieldDoesNotBlockWhileATimerIsRegisteredAndHasYetToEl new AsyncSinkWriterImplBuilder().context(sinkInitContext).build(); sink.write(String.valueOf(0)); sink.flush(true); - assertEquals(1, res.size()); + + assertThat(res.size()).isEqualTo(1); } @Test @@ -162,12 +168,14 @@ public void testThatSnapshotsAreTakenOfBufferCorrectlyBeforeAndAfterAutomaticFlu sink.write("25"); sink.write("55"); + assertThatBufferStatesAreEqual(sink.wrapRequests(25, 55), getWriterState(sink)); - assertEquals(0, res.size()); + assertThat(res.size()).isEqualTo(0); sink.write("75"); + assertThatBufferStatesAreEqual(BufferedRequestState.emptyState(), getWriterState(sink)); - assertEquals(3, res.size()); + assertThat(res.size()).isEqualTo(3); } public void writeFiveRecordsWithOneFailingThenCallPrepareCommitWithFlushing() @@ -180,6 +188,7 @@ public void writeFiveRecordsWithOneFailingThenCallPrepareCommitWithFlushing() sink.write("75"); sink.write("95"); sink.write("955"); + assertThatBufferStatesAreEqual(sink.wrapRequests(95, 955), getWriterState(sink)); sink.flush(true); assertThatBufferStatesAreEqual(BufferedRequestState.emptyState(), getWriterState(sink)); @@ -189,15 +198,17 @@ public void writeFiveRecordsWithOneFailingThenCallPrepareCommitWithFlushing() public void testThatSnapshotsAreTakenOfBufferCorrectlyBeforeAndAfterManualFlush() throws IOException, InterruptedException { writeFiveRecordsWithOneFailingThenCallPrepareCommitWithFlushing(); - assertEquals(5, res.size()); + + assertThat(res.size()).isEqualTo(5); } @Test public void metricsAreLoggedEachTimeSubmitRequestEntriesIsCalled() throws IOException, InterruptedException { writeFiveRecordsWithOneFailingThenCallPrepareCommitWithFlushing(); - assertEquals(5, sinkInitContext.getNumRecordsOutCounter().getCount()); - assertEquals(20, sinkInitContext.getNumBytesOutCounter().getCount()); + + assertThat(sinkInitContext.getNumRecordsOutCounter().getCount()).isEqualTo(5); + assertThat(sinkInitContext.getNumBytesOutCounter().getCount()).isEqualTo(20); } @Test @@ -215,11 +226,11 @@ public void testRuntimeErrorsInSubmitRequestEntriesEndUpAsIOExceptionsWithNumOfF sink.write("75"); sink.write("95"); sink.write("35"); - Exception e = assertThrows(RuntimeException.class, () -> sink.write("135")); - assertEquals( - "Deliberate runtime exception occurred in SinkWriterImplementation.", - e.getMessage()); - assertEquals(3, res.size()); + + assertThatThrownBy(() -> sink.write("135")) + .isInstanceOf(RuntimeException.class) + .hasMessage("Deliberate runtime exception occurred in SinkWriterImplementation."); + assertThat(res.size()).isEqualTo(3); } @Test @@ -292,8 +303,8 @@ public void testRetryableErrorsDoNotViolateAtLeastOnceSemanticsDueToRequeueOfFai sink.flush(true); // Everything is saved - assertEquals(Arrays.asList(25, 55, 965, 75, 95, 955, 550, 45, 35, 535), res); - assertEquals(0, getWriterState(sink).getStateSize()); + assertThat(res).isEqualTo(Arrays.asList(25, 55, 965, 75, 95, 955, 550, 45, 35, 535)); + assertThat(getWriterState(sink).getStateSize()).isEqualTo(0); } @Test @@ -342,40 +353,39 @@ private void testFailedEntriesAreRetriedInTheNextPossiblePersistRequestAndNoLate // Buffer continues to fill up without blocking on write, until eventually yield is called // on the mailbox thread during the prepare commit sink.flush(true); - assertEquals(Arrays.asList(25, 55, 965, 75, 95, 955, 550, 645, 545, 535, 515, 505), res); + + assertThat(res) + .isEqualTo(Arrays.asList(25, 55, 965, 75, 95, 955, 550, 645, 545, 535, 515, 505)); } @Test public void testThatMaxBufferSizeOfSinkShouldBeStrictlyGreaterThanMaxSizeOfEachBatch() { - Exception e = - assertThrows( - IllegalArgumentException.class, + assertThatThrownBy( () -> new AsyncSinkWriterImplBuilder() .context(sinkInitContext) .maxBufferedRequests(10) - .build()); - assertEquals( - "The maximum number of requests that may be buffered should be " - + "strictly greater than the maximum number of requests per batch.", - e.getMessage()); + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "The maximum number of requests that may be buffered should be " + + "strictly greater than the maximum number of requests per batch."); } @Test public void maxRecordSizeSetMustBeSmallerThanOrEqualToMaxBatchSize() { - Exception e = - assertThrows( - IllegalArgumentException.class, + assertThatThrownBy( () -> new AsyncSinkWriterImplBuilder() .context(sinkInitContext) .maxBufferedRequests(11) .maxBatchSizeInBytes(10_000) .maxRecordSizeInBytes(10_001) - .build()); - assertEquals( - "The maximum allowed size in bytes per flush must be greater than or equal to the maximum allowed size in bytes of a single record.", - e.getMessage()); + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "The maximum allowed size in bytes per flush must be greater than or equal to" + + " the maximum allowed size in bytes of a single record."); } @Test @@ -388,17 +398,20 @@ public void recordsWrittenToTheSinkMustBeSmallerOrEqualToMaxRecordSizeInBytes() .maxBatchSizeInBytes(10_000) .maxRecordSizeInBytes(3) .build(); - Exception e = assertThrows(IllegalArgumentException.class, () -> sink.write("3")); - assertEquals( - "The request entry sent to the buffer was of size [4], when the maxRecordSizeInBytes was set to [3].", - e.getMessage()); + + assertThatThrownBy(() -> sink.write("3")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "The request entry sent to the buffer was of size [4], when " + + "the maxRecordSizeInBytes was set to [3]."); } private void writeXToSinkAssertDestinationIsInStateYAndBufferHasZ( AsyncSinkWriterImpl sink, String x, List y, List z) throws IOException, InterruptedException { sink.write(x); - assertEquals(y, res); + + assertThat(res).isEqualTo(y); assertThatBufferStatesAreEqual(sink.wrapRequests(z), getWriterState(sink)); } @@ -418,7 +431,8 @@ public void testFlushThresholdMetBeforeBatchLimitWillCreateASmallerBatchOfSizeAb */ for (int i = 0; i < 100; i++) { sink.write(String.valueOf(i)); - assertEquals((i / 7) * 7, res.size()); + + assertThat(res.size()).isEqualTo((i / 7) * 7); } } @@ -430,7 +444,8 @@ public void prepareCommitDoesNotFlushElementsIfFlushIsSetToFalse() throws Except sink.write(String.valueOf(1)); sink.write(String.valueOf(2)); sink.flush(false); - assertEquals(0, res.size()); + + assertThat(res.size()).isEqualTo(0); } @Test @@ -447,11 +462,14 @@ public void testThatWhenNumberOfItemAndSizeOfRecordThresholdsAreMetSimultaneousl for (int i = 0; i < 7; i++) { sink.write(String.valueOf(i)); } - assertEquals(7, res.size()); + + assertThat(res.size()).isEqualTo(7); + for (int i = 7; i < 14; i++) { sink.write(String.valueOf(i)); } - assertEquals(14, res.size()); + + assertThat(res.size()).isEqualTo(14); } @Test @@ -468,13 +486,17 @@ public void testThatIntermittentlyFailingEntriesAreEnqueuedOnToTheBufferWithCorr sink.write(String.valueOf(1)); // Buffer: 104/110B; 2/10 elements; 0 inflight sink.write(String.valueOf(2)); // Buffer: 108/110B; 3/10 elements; 0 inflight sink.write(String.valueOf(3)); // Buffer: 112/110B; 4/10 elements; 0 inflight -- flushing - assertEquals(2, res.size()); // Request was [225, 1, 2], element 225 failed on first attempt + + assertThat(res.size()) + .isEqualTo(2); // Request was [225, 1, 2], element 225 failed on first attempt + sink.write(String.valueOf(4)); // Buffer: 8/110B; 2/10 elements; 1 inflight sink.write(String.valueOf(5)); // Buffer: 12/110B; 3/10 elements; 1 inflight sink.write(String.valueOf(6)); // Buffer: 16/110B; 4/10 elements; 1 inflight sink.write(String.valueOf(325)); // Buffer: 116/110B; 5/10 elements; 1 inflight -- flushing + // inflight request is processed, buffer: [225, 3, 4, 5, 6, 325] - assertEquals(Arrays.asList(1, 2, 225, 3, 4), res); + assertThat(res).isEqualTo(Arrays.asList(1, 2, 225, 3, 4)); // Buffer: [5, 6, 325]; 0 inflight } @@ -494,14 +516,18 @@ public void testThatIntermittentlyFailingEntriesAreEnqueuedOnToTheBufferWithCorr sink.write(String.valueOf(1)); // Buffer: 204/210B; 3/10 elements; 0 inflight sink.write(String.valueOf(2)); // Buffer: 208/210B; 4/10 elements; 0 inflight sink.write(String.valueOf(3)); // Buffer: 212/210B; 5/10 elements; 0 inflight -- flushing - assertEquals(2, res.size()); // Request was [228, 225, 1, 2], element 228, 225 failed + + assertThat(res.size()) + .isEqualTo(2); // Request was [228, 225, 1, 2], element 228, 225 failed + sink.write(String.valueOf(4)); // Buffer: 8/210B; 2/10 elements; 2 inflight sink.write(String.valueOf(5)); // Buffer: 12/210B; 3/10 elements; 2 inflight sink.write(String.valueOf(6)); // Buffer: 16/210B; 4/10 elements; 2 inflight sink.write(String.valueOf(328)); // Buffer: 116/210B; 5/10 elements; 2 inflight sink.write(String.valueOf(325)); // Buffer: 216/210B; 6/10 elements; 2 inflight -- flushing + // inflight request is processed, buffer: [228, 225, 3, 4, 5, 6, 328, 325] - assertEquals(Arrays.asList(1, 2, 228, 225, 3, 4), res); + assertThat(res).isEqualTo(Arrays.asList(1, 2, 228, 225, 3, 4)); // Buffer: [5, 6, 328, 325]; 0 inflight } @@ -526,9 +552,12 @@ public void testThatABatchWithSizeSmallerThanMaxBatchSizeIsFlushedOnTimeoutExpir } tpts.setCurrentTime(99L); - assertEquals(0, res.size()); + + assertThat(res.size()).isEqualTo(0); + tpts.setCurrentTime(100L); - assertEquals(8, res.size()); + + assertThat(res.size()).isEqualTo(8); } @Test @@ -551,9 +580,9 @@ public void testThatTimeBasedBatchPicksUpAllRelevantItemsUpUntilExpiryOfTimer() sink.write(String.valueOf(i)); } tpts.setCurrentTime(99L); - assertEquals(90, res.size()); + assertThat(res.size()).isEqualTo(90); tpts.setCurrentTime(100L); - assertEquals(98, res.size()); + assertThat(res.size()).isEqualTo(98); } @Test @@ -571,15 +600,17 @@ public void prepareCommitFlushesInflightElementsAndDoesNotFlushIfFlushIsSetToFal sink.write(String.valueOf(1)); // buffer: [225, 0, 1] sink.write(String.valueOf(2)); // buffer: [2], inflight: [225], destination: [0, 1] - assertEquals(Arrays.asList(0, 1), res); + assertThat(res).isEqualTo(Arrays.asList(0, 1)); assertThatBufferStatesAreEqual(sink.wrapRequests(2), getWriterState(sink)); sink.flush(false); // buffer: [225, 2], inflight: [], destination: [0, 1] - assertEquals(Arrays.asList(0, 1), res); + + assertThat(res).isEqualTo(Arrays.asList(0, 1)); assertThatBufferStatesAreEqual(sink.wrapRequests(225, 2), getWriterState(sink)); sink.flush(true); // buffer: [], inflight: [], destination: [0, 1, 225, 2] - assertEquals(Arrays.asList(0, 1, 225, 2), res); + + assertThat(res).isEqualTo(Arrays.asList(0, 1, 225, 2)); } @Test @@ -596,11 +627,14 @@ public void testThatIntermittentlyFailingEntriesAreEnqueuedOnToTheBufferAfterSna sink.write(String.valueOf(1)); // Buffer: 104/110B; 3/10 elements; 0 inflight sink.write(String.valueOf(2)); // Buffer: 108/110B; 4/10 elements; 0 inflight sink.write(String.valueOf(3)); // Buffer: 112/110B; 5/10 elements; 0 inflight -- flushing - assertEquals(2, res.size()); // Request was [225, 1, 2], element 225 failed + + assertThat(res.size()).isEqualTo(2); // Request was [225, 1, 2], element 225 failed // buffer should be [3] with [225] inflight sink.flush(false); // Buffer: [225,3] - > 8/110; 2/10 elements; 0 inflight - assertEquals(2, res.size()); // + + assertThat(res.size()).isEqualTo(2); // + List> states = sink.snapshotState(1); AsyncSinkWriterImpl newSink = new AsyncSinkWriterImplBuilder() @@ -610,7 +644,8 @@ public void testThatIntermittentlyFailingEntriesAreEnqueuedOnToTheBufferAfterSna newSink.write(String.valueOf(4)); // Buffer: 12/15B; 3/10 elements; 0 inflight newSink.write(String.valueOf(5)); // Buffer: 16/15B; 4/10 elements; 0 inflight --flushing - assertEquals(Arrays.asList(1, 2, 225, 3, 4), res); + + assertThat(res).isEqualTo(Arrays.asList(1, 2, 225, 3, 4)); // Buffer: [5]; 0 inflight } @@ -687,20 +722,20 @@ public void testThatOneAndOnlyOneCallbackIsEverRegistered() throws Exception { TestProcessingTimeService tpts = sinkInitContext.getTestProcessingTimeService(); tpts.setCurrentTime(0L); sink.write("1"); // A timer is registered here to elapse at t=100 - assertEquals(0, res.size()); + assertThat(res.size()).isEqualTo(0); tpts.setCurrentTime(10L); sink.flush(true); - assertEquals(1, res.size()); + assertThat(res.size()).isEqualTo(1); tpts.setCurrentTime(20L); // At t=20, we write a new element that should not trigger another sink.write("2"); // timer to be registered. If it is, it should elapse at t=120s. - assertEquals(1, res.size()); + assertThat(res.size()).isEqualTo(1); tpts.setCurrentTime(100L); - assertEquals(2, res.size()); + assertThat(res.size()).isEqualTo(2); sink.write("3"); tpts.setCurrentTime(199L); // At t=199s, our third element has not been written - assertEquals(2, res.size()); // therefore, no timer fired at 120s. + assertThat(res.size()).isEqualTo(2); // therefore, no timer fired at 120s. tpts.setCurrentTime(200L); - assertEquals(3, res.size()); + assertThat(res.size()).isEqualTo(3); } @Test @@ -721,13 +756,13 @@ public void testThatIntermittentlyFailingEntriesShouldBeFlushedWithMainBatchInTi sink.write("2"); sink.write("225"); tpts.setCurrentTime(100L); - assertEquals(2, res.size()); + assertThat(res.size()).isEqualTo(2); sink.write("3"); sink.write("4"); tpts.setCurrentTime(199L); - assertEquals(2, res.size()); + assertThat(res.size()).isEqualTo(2); tpts.setCurrentTime(200L); - assertEquals(5, res.size()); + assertThat(res.size()).isEqualTo(5); } @Test @@ -748,7 +783,7 @@ public void testThatFlushingAnEmptyBufferDoesNotResultInErrorOrFailure() throws sink.write("1"); tpts.setCurrentTime(50L); sink.flush(true); - assertEquals(1, res.size()); + assertThat(res.size()).isEqualTo(1); tpts.setCurrentTime(200L); } @@ -769,10 +804,10 @@ public void testThatOnExpiryOfAnOldTimeoutANewOneMayBeRegisteredImmediately() th tpts.setCurrentTime(0L); sink.write("1"); tpts.setCurrentTime(100L); - assertEquals(1, res.size()); + assertThat(res.size()).isEqualTo(1); sink.write("2"); tpts.setCurrentTime(200L); - assertEquals(2, res.size()); + assertThat(res.size()).isEqualTo(2); } /** @@ -806,7 +841,7 @@ public void testThatInterleavingThreadsMayBlockEachOtherButDoNotCauseRaceConditi true); writeTwoElementsAndInterleaveTheNextTwoElements(sink, blockedWriteLatch, delayedStartLatch); - assertEquals(Arrays.asList(1, 2, 3, 4), res); + assertThat(res).isEqualTo(Arrays.asList(1, 2, 3, 4)); } /** @@ -837,7 +872,7 @@ public void testThatIfOneInterleavedThreadIsBlockedTheOtherThreadWillContinueAnd false); writeTwoElementsAndInterleaveTheNextTwoElements(sink, blockedWriteLatch, delayedStartLatch); - assertEquals(Arrays.asList(4, 1, 2, 3), res); + assertThat(res).isEqualTo(Arrays.asList(4, 1, 2, 3)); } private void writeTwoElementsAndInterleaveTheNextTwoElements( @@ -866,9 +901,9 @@ private void writeTwoElementsAndInterleaveTheNextTwoElements( tpts.setCurrentTime(100L); blockedWriteLatch.countDown(); es.shutdown(); - assertTrue( - es.awaitTermination(500, TimeUnit.MILLISECONDS), - "Executor Service stuck at termination, not terminated after 500ms!"); + assertThat(es.awaitTermination(500, TimeUnit.MILLISECONDS)) + .as("Executor Service stuck at termination, not terminated after 500ms!") + .isTrue(); } /** @@ -924,19 +959,19 @@ public void ifTheNumberOfUncompletedInFlightRequestsIsTooManyThenBlockInFlushMet } }); Thread.sleep(300); - assertFalse(s.isInterrupted()); + assertThat(s.isInterrupted()).isFalse(); s.interrupt(); blockedWriteLatch.countDown(); t.join(); - assertEquals(Arrays.asList(1, 2, 3), res); + assertThat(res).isEqualTo(Arrays.asList(1, 2, 3)); } private BufferedRequestState getWriterState( AsyncSinkWriter sinkWriter) { List> states = sinkWriter.snapshotState(1); - assertEquals(states.size(), 1); + assertThat(states.size()).isEqualTo(1); return states.get(0); } @@ -1205,10 +1240,11 @@ protected void submitRequestEntries( try { delayedStartLatch.countDown(); if (blockForLimitedTime) { - assertFalse( - blockedThreadLatch.await(500, TimeUnit.MILLISECONDS), - "The countdown latch was released before the full amount" - + "of time was reached."); + assertThat(blockedThreadLatch.await(500, TimeUnit.MILLISECONDS)) + .as( + "The countdown latch was released before the full amount" + + "of time was reached.") + .isFalse(); } else { blockedThreadLatch.await(); } diff --git a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/writer/TestSinkInitContext.java b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/writer/TestSinkInitContext.java index b1461903fc829..a7e4979efdf37 100644 --- a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/writer/TestSinkInitContext.java +++ b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/sink/writer/TestSinkInitContext.java @@ -146,10 +146,10 @@ public Optional> getCurrentSendTimeGauge() { } public Counter getNumRecordsOutCounter() { - return metricGroup.getIOMetricGroup().getNumRecordsOutCounter(); + return metricGroup.getNumRecordsSendCounter(); } public Counter getNumBytesOutCounter() { - return metricGroup.getIOMetricGroup().getNumBytesOutCounter(); + return metricGroup.getNumBytesSendCounter(); } } diff --git a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/SourceReaderBaseTest.java b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/SourceReaderBaseTest.java index 4a5544d414802..d17e1cfcf8756 100644 --- a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/SourceReaderBaseTest.java +++ b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/SourceReaderBaseTest.java @@ -18,10 +18,14 @@ package org.apache.flink.connector.base.source.reader; +import org.apache.flink.api.common.eventtime.WatermarkGenerator; +import org.apache.flink.api.common.eventtime.WatermarkOutput; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.SourceReader; import org.apache.flink.api.connector.source.SourceSplit; import org.apache.flink.api.connector.source.mocks.MockSourceSplit; +import org.apache.flink.api.connector.source.mocks.MockSourceSplitSerializer; import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.source.reader.fetcher.SingleThreadFetcherManager; import org.apache.flink.connector.base.source.reader.mocks.MockSourceReader; @@ -37,9 +41,22 @@ import org.apache.flink.connector.testutils.source.reader.TestingReaderContext; import org.apache.flink.connector.testutils.source.reader.TestingReaderOutput; import org.apache.flink.core.io.InputStatus; +import org.apache.flink.core.testutils.CommonTestUtils; +import org.apache.flink.runtime.source.event.AddSplitEvent; +import org.apache.flink.streaming.api.operators.SourceOperator; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.io.PushingAsyncDataInput; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -48,12 +65,15 @@ import java.util.concurrent.CompletableFuture; import java.util.function.Supplier; +import static org.apache.flink.streaming.api.operators.source.TestingSourceOperator.createTestOperator; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; /** A unit test class for {@link SourceReaderBase}. */ public class SourceReaderBaseTest extends SourceReaderTestBase { + private static final Logger LOG = LoggerFactory.getLogger(SourceReaderBaseTest.class); + @Test void testExceptionInSplitReader() { assertThatThrownBy( @@ -239,6 +259,83 @@ void testPollNextReturnMoreAvailableWhenAllSplitFetcherCloseWithLeftoverElementI .isEqualTo(InputStatus.MORE_AVAILABLE); } + @ParameterizedTest(name = "Emit record before split addition: {0}") + @ValueSource(booleans = {true, false}) + void testPerSplitWatermark(boolean emitRecordBeforeSplitAddition) throws Exception { + MockSplitReader mockSplitReader = + MockSplitReader.newBuilder() + .setNumRecordsPerSplitPerFetch(3) + .setBlockingFetch(true) + .build(); + + MockSourceReader reader = + new MockSourceReader( + new FutureCompletingBlockingQueue<>(), + () -> mockSplitReader, + new Configuration(), + new TestingReaderContext()); + + SourceOperator sourceOperator = + createTestOperator( + reader, + WatermarkStrategy.forGenerator( + (context) -> new OnEventWatermarkGenerator()), + true); + + MockSourceSplit splitA = new MockSourceSplit(0, 0, 3); + splitA.addRecord(100); + splitA.addRecord(200); + splitA.addRecord(300); + + MockSourceSplit splitB = new MockSourceSplit(1, 0, 3); + splitB.addRecord(150); + splitB.addRecord(250); + splitB.addRecord(350); + + WatermarkCollectingDataOutput output = new WatermarkCollectingDataOutput(); + + if (emitRecordBeforeSplitAddition) { + sourceOperator.emitNext(output); + } + + AddSplitEvent addSplitsEvent = + new AddSplitEvent<>(Arrays.asList(splitA, splitB), new MockSourceSplitSerializer()); + sourceOperator.handleOperatorEvent(addSplitsEvent); + + // First 3 records from split A should not generate any watermarks + CommonTestUtils.waitUtil( + () -> { + try { + sourceOperator.emitNext(output); + } catch (Exception e) { + LOG.warn("Exception caught at emitting records", e); + return false; + } + return output.numRecords == 3; + }, + Duration.ofSeconds(10), + String.format( + "%d out of 3 records are received within timeout", output.numRecords)); + assertThat(output.watermarks).isEmpty(); + + CommonTestUtils.waitUtil( + () -> { + try { + sourceOperator.emitNext(output); + } catch (Exception e) { + LOG.warn("Exception caught at emitting records", e); + return false; + } + return output.numRecords == 6; + }, + Duration.ofSeconds(10), + String.format( + "%d out of 6 records are received within timeout", output.numRecords)); + + assertThat(output.watermarks).hasSize(3); + assertThat(output.watermarks).containsExactly(150L, 250L, 300L); + } + // ---------------- helper methods ----------------- @Override @@ -379,4 +476,37 @@ private void shutdownAllSplitFetcher() { } } } + + private static class OnEventWatermarkGenerator implements WatermarkGenerator { + + @Override + public void onEvent(Integer event, long eventTimestamp, WatermarkOutput output) { + output.emitWatermark(new org.apache.flink.api.common.eventtime.Watermark(event)); + } + + @Override + public void onPeriodicEmit(WatermarkOutput output) {} + } + + private static class WatermarkCollectingDataOutput + implements PushingAsyncDataInput.DataOutput { + int numRecords = 0; + final List watermarks = new ArrayList<>(); + + @Override + public void emitRecord(StreamRecord streamRecord) { + numRecords++; + } + + @Override + public void emitWatermark(Watermark watermark) throws Exception { + watermarks.add(watermark.getTimestamp()); + } + + @Override + public void emitWatermarkStatus(WatermarkStatus watermarkStatus) {} + + @Override + public void emitLatencyMarker(LatencyMarker latencyMarker) {} + } } diff --git a/flink-connectors/flink-connector-cassandra/pom.xml b/flink-connectors/flink-connector-cassandra/pom.xml index b03ed1b4e4c05..e44a12c0d0f3e 100644 --- a/flink-connectors/flink-connector-cassandra/pom.xml +++ b/flink-connectors/flink-connector-cassandra/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. @@ -205,13 +205,6 @@ under the License. cassandra test - - - io.netty - netty-all - 4.1.46.Final - test - diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java index 924c6846ace2f..de14e861ccb20 100644 --- a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java +++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java @@ -62,7 +62,6 @@ import net.bytebuddy.ByteBuddy; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; @@ -165,9 +164,6 @@ protected Cluster buildCluster(Cluster.Builder builder) { "CREATE KEYSPACE " + KEYSPACE + " WITH replication= {'class':'SimpleStrategy', 'replication_factor':1};"; - private static final String DROP_KEYSPACE_QUERY = "DROP KEYSPACE IF EXISTS " + KEYSPACE + " ;"; - private static final String DROP_TABLE_QUERY = - "DROP TABLE IF EXISTS " + KEYSPACE + "." + TABLE_NAME_VARIABLE + " ;"; private static final String CREATE_TABLE_QUERY = "CREATE TABLE " + KEYSPACE @@ -386,10 +382,7 @@ public static void startAndInitializeCassandra() { } } } - session.execute(DROP_KEYSPACE_QUERY); session.execute(CREATE_KEYSPACE_QUERY); - session.execute( - CREATE_TABLE_QUERY.replace(TABLE_NAME_VARIABLE, TABLE_NAME_PREFIX + "initial")); } @Before @@ -398,14 +391,6 @@ public void createTable() { session.execute(injectTableName(CREATE_TABLE_QUERY)); } - @After - public void dropTables() { - // need to drop tables in case of retrials. Need to drop all the tables - // that are created in test because this method is executed with every test - session.execute(DROP_KEYSPACE_QUERY); - session.execute(CREATE_KEYSPACE_QUERY); - } - @AfterClass public static void closeCassandra() { if (session != null) { @@ -722,9 +707,7 @@ builderForWriting, injectTableName(INSERT_DATA_QUERY)) private static int retrialsCount = 0; @Test - public void testRetrialAndDropTables() { - // should not fail with table exists upon retrial - // as @After method that truncate the keyspace is called upon retrials. + public void testRetrial() { annotatePojoWithTable(KEYSPACE, TABLE_NAME_PREFIX + tableID); if (retrialsCount < 2) { retrialsCount++; diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index cd2c6c5973e6b..744a8decc4a2e 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java index 30ee238b1a5e1..efe6dc2421cc8 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java @@ -19,6 +19,7 @@ package org.apache.flink.connector.elasticsearch.sink; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.api.connector.sink2.SinkWriter; import org.apache.flink.connector.base.DeliveryGuarantee; @@ -88,4 +89,9 @@ public SinkWriter createWriter(InitContext context) throws IOException { context.metricGroup(), context.getMailboxExecutor()); } + + @VisibleForTesting + DeliveryGuarantee getDeliveryGuarantee() { + return deliveryGuarantee; + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java index fe64c94902b47..8543255ddfec4 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java @@ -48,7 +48,7 @@ public abstract class ElasticsearchSinkBuilderBase< private FlushBackoffType bulkFlushBackoffType = FlushBackoffType.NONE; private int bulkFlushBackoffRetries = -1; private long bulkFlushBackOffDelay = -1; - private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.NONE; + private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE; private List hosts; protected ElasticsearchEmitter emitter; private String username; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java index 7ea3dd87bd5a4..fa8ed67cb0785 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java @@ -103,7 +103,7 @@ class ElasticsearchWriter implements SinkWriter { RestClient.builder(hosts.toArray(new HttpHost[0])), networkClientConfig)); this.bulkProcessor = createBulkProcessor(bulkProcessorBuilderFactory, bulkProcessorConfig); - this.requestIndexer = new DefaultRequestIndexer(); + this.requestIndexer = new DefaultRequestIndexer(metricGroup.getNumRecordsSendCounter()); checkNotNull(metricGroup); metricGroup.setCurrentSendTimeGauge(() -> ackTime - lastSendTime); this.numBytesOutCounter = metricGroup.getIOMetricGroup().getNumBytesOutCounter(); @@ -294,9 +294,16 @@ private boolean isClosed() { private class DefaultRequestIndexer implements RequestIndexer { + private final Counter numRecordsSendCounter; + + public DefaultRequestIndexer(Counter numRecordsSendCounter) { + this.numRecordsSendCounter = checkNotNull(numRecordsSendCounter); + } + @Override public void add(DeleteRequest... deleteRequests) { for (final DeleteRequest deleteRequest : deleteRequests) { + numRecordsSendCounter.inc(); pendingActions++; bulkProcessor.add(deleteRequest); } @@ -305,6 +312,7 @@ public void add(DeleteRequest... deleteRequests) { @Override public void add(IndexRequest... indexRequests) { for (final IndexRequest indexRequest : indexRequests) { + numRecordsSendCounter.inc(); pendingActions++; bulkProcessor.add(indexRequest); } @@ -313,6 +321,7 @@ public void add(IndexRequest... indexRequests) { @Override public void add(UpdateRequest... updateRequests) { for (final UpdateRequest updateRequest : updateRequests) { + numRecordsSendCounter.inc(); pendingActions++; bulkProcessor.add(updateRequest); } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConnectorOptions.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConnectorOptions.java index 672f07274cff8..10ea0ae2a48bb 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConnectorOptions.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConnectorOptions.java @@ -143,6 +143,6 @@ public class ElasticsearchConnectorOptions { public static final ConfigOption DELIVERY_GUARANTEE_OPTION = ConfigOptions.key("sink.delivery-guarantee") .enumType(DeliveryGuarantee.class) - .defaultValue(DeliveryGuarantee.NONE) + .defaultValue(DeliveryGuarantee.AT_LEAST_ONCE) .withDescription("Optional delivery guarantee when committing."); } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java index ec2a00693658a..92886f40e0932 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java @@ -33,7 +33,6 @@ import java.time.LocalDateTime; import java.time.LocalTime; import java.time.ZoneId; -import java.time.ZoneOffset; import java.time.format.DateTimeFormatter; import java.util.ArrayList; import java.util.Arrays; @@ -133,7 +132,8 @@ public String generate(RowData row) { final String dateTimeFormat = indexHelper.extractDateFormat(index, indexFieldLogicalTypeRoot); DynamicFormatter formatFunction = - createFormatFunction(indexFieldType, indexFieldLogicalTypeRoot); + createFormatFunction( + indexFieldType, indexFieldLogicalTypeRoot, localTimeZoneId); return new AbstractTimeIndexGenerator(index, dateTimeFormat) { @Override @@ -163,7 +163,9 @@ public String generate(RowData row) { } private static DynamicFormatter createFormatFunction( - LogicalType indexFieldType, LogicalTypeRoot indexFieldLogicalTypeRoot) { + LogicalType indexFieldType, + LogicalTypeRoot indexFieldLogicalTypeRoot, + ZoneId localTimeZoneId) { switch (indexFieldLogicalTypeRoot) { case DATE: return (value, dateTimeFormatter) -> { @@ -186,7 +188,7 @@ private static DynamicFormatter createFormatFunction( case TIMESTAMP_WITH_LOCAL_TIME_ZONE: return (value, dateTimeFormatter) -> { TimestampData indexField = (TimestampData) value; - return indexField.toInstant().atZone(ZoneOffset.UTC).format(dateTimeFormatter); + return indexField.toInstant().atZone(localTimeZoneId).format(dateTimeFormatter); }; default: throw new TableException( diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.java new file mode 100644 index 0000000000000..6c22cf3bfbed8 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; + +import java.time.format.DateTimeFormatter; + +/** Abstract class for time related {@link IndexGenerator}. */ +@Internal +abstract class AbstractTimeIndexGenerator extends IndexGeneratorBase { + + private final String dateTimeFormat; + protected transient DateTimeFormatter dateTimeFormatter; + + public AbstractTimeIndexGenerator(String index, String dateTimeFormat) { + super(index); + this.dateTimeFormat = dateTimeFormat; + } + + @Override + public void open() { + this.dateTimeFormatter = DateTimeFormatter.ofPattern(dateTimeFormat); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java new file mode 100644 index 0000000000000..04c76333d5353 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.util.IgnoringFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.util.RetryRejectedExecutionFailureHandler; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.util.InstantiationUtil; + +import java.time.Duration; +import java.util.Objects; +import java.util.Optional; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; + +/** Accessor methods to elasticsearch options. */ +@Internal +class ElasticsearchConfiguration { + protected final ReadableConfig config; + private final ClassLoader classLoader; + + ElasticsearchConfiguration(ReadableConfig config, ClassLoader classLoader) { + this.config = config; + this.classLoader = classLoader; + } + + public ActionRequestFailureHandler getFailureHandler() { + final ActionRequestFailureHandler failureHandler; + String value = config.get(FAILURE_HANDLER_OPTION); + switch (value.toUpperCase()) { + case "FAIL": + failureHandler = new NoOpFailureHandler(); + break; + case "IGNORE": + failureHandler = new IgnoringFailureHandler(); + break; + case "RETRY-REJECTED": + failureHandler = new RetryRejectedExecutionFailureHandler(); + break; + default: + try { + Class failureHandlerClass = Class.forName(value, false, classLoader); + failureHandler = + (ActionRequestFailureHandler) + InstantiationUtil.instantiate(failureHandlerClass); + } catch (ClassNotFoundException e) { + throw new ValidationException( + "Could not instantiate the failure handler class: " + value, e); + } + break; + } + return failureHandler; + } + + public String getDocumentType() { + return config.get(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION); + } + + public int getBulkFlushMaxActions() { + int maxActions = config.get(ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION); + // convert 0 to -1, because Elasticsearch client use -1 to disable this configuration. + return maxActions == 0 ? -1 : maxActions; + } + + public long getBulkFlushMaxByteSize() { + long maxSize = + config.get(ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION).getBytes(); + // convert 0 to -1, because Elasticsearch client use -1 to disable this configuration. + return maxSize == 0 ? -1 : maxSize; + } + + public long getBulkFlushInterval() { + long interval = config.get(BULK_FLUSH_INTERVAL_OPTION).toMillis(); + // convert 0 to -1, because Elasticsearch client use -1 to disable this configuration. + return interval == 0 ? -1 : interval; + } + + public Optional getUsername() { + return config.getOptional(USERNAME_OPTION); + } + + public Optional getPassword() { + return config.getOptional(PASSWORD_OPTION); + } + + public boolean isBulkFlushBackoffEnabled() { + return config.get(BULK_FLUSH_BACKOFF_TYPE_OPTION) + != ElasticsearchConnectorOptions.BackOffType.DISABLED; + } + + public Optional getBulkFlushBackoffType() { + switch (config.get(BULK_FLUSH_BACKOFF_TYPE_OPTION)) { + case CONSTANT: + return Optional.of(ElasticsearchSinkBase.FlushBackoffType.CONSTANT); + case EXPONENTIAL: + return Optional.of(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); + default: + return Optional.empty(); + } + } + + public Optional getBulkFlushBackoffRetries() { + return config.getOptional(BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION); + } + + public Optional getBulkFlushBackoffDelay() { + return config.getOptional(BULK_FLUSH_BACKOFF_DELAY_OPTION).map(Duration::toMillis); + } + + public boolean isDisableFlushOnCheckpoint() { + return !config.get(ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION); + } + + public String getIndex() { + return config.get(ElasticsearchConnectorOptions.INDEX_OPTION); + } + + public String getKeyDelimiter() { + return config.get(ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION); + } + + public Optional getPathPrefix() { + return config.getOptional(ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ElasticsearchConfiguration that = (ElasticsearchConfiguration) o; + return Objects.equals(config, that.config) && Objects.equals(classLoader, that.classLoader); + } + + @Override + public int hashCode() { + return Objects.hash(config, classLoader); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java new file mode 100644 index 0000000000000..38d65a7fba62b --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java @@ -0,0 +1,165 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.description.Description; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; + +import java.time.Duration; +import java.util.List; + +import static org.apache.flink.configuration.description.TextElement.text; + +/** Options for the Elasticsearch connector. */ +@PublicEvolving +public class ElasticsearchConnectorOptions { + + public static final ConfigOption> HOSTS_OPTION = + ConfigOptions.key("hosts") + .stringType() + .asList() + .noDefaultValue() + .withDescription("Elasticsearch hosts to connect to."); + + public static final ConfigOption INDEX_OPTION = + ConfigOptions.key("index") + .stringType() + .noDefaultValue() + .withDescription("Elasticsearch index for every record."); + + public static final ConfigOption DOCUMENT_TYPE_OPTION = + ConfigOptions.key("document-type") + .stringType() + .noDefaultValue() + .withDescription("Elasticsearch document type."); + + public static final ConfigOption PASSWORD_OPTION = + ConfigOptions.key("password") + .stringType() + .noDefaultValue() + .withDescription("Password used to connect to Elasticsearch instance."); + + public static final ConfigOption USERNAME_OPTION = + ConfigOptions.key("username") + .stringType() + .noDefaultValue() + .withDescription("Username used to connect to Elasticsearch instance."); + + public static final ConfigOption KEY_DELIMITER_OPTION = + ConfigOptions.key("document-id.key-delimiter") + .stringType() + .defaultValue("_") + .withDescription( + "Delimiter for composite keys e.g., \"$\" would result in IDs \"KEY1$KEY2$KEY3\"."); + + public static final ConfigOption FAILURE_HANDLER_OPTION = + ConfigOptions.key("failure-handler") + .stringType() + .defaultValue("fail") + .withDescription( + Description.builder() + .text( + "Failure handling strategy in case a request to Elasticsearch fails") + .list( + text( + "\"fail\" (throws an exception if a request fails and thus causes a job failure)"), + text( + "\"ignore\" (ignores failures and drops the request)"), + text( + "\"retry-rejected\" (re-adds requests that have failed due to queue capacity saturation)"), + text( + "\"class name\" for failure handling with a ActionRequestFailureHandler subclass")) + .build()); + + public static final ConfigOption FLUSH_ON_CHECKPOINT_OPTION = + ConfigOptions.key("sink.flush-on-checkpoint") + .booleanType() + .defaultValue(true) + .withDescription("Disables flushing on checkpoint"); + + public static final ConfigOption BULK_FLUSH_MAX_ACTIONS_OPTION = + ConfigOptions.key("sink.bulk-flush.max-actions") + .intType() + .defaultValue(1000) + .withDescription("Maximum number of actions to buffer for each bulk request."); + + public static final ConfigOption BULK_FLASH_MAX_SIZE_OPTION = + ConfigOptions.key("sink.bulk-flush.max-size") + .memoryType() + .defaultValue(MemorySize.parse("2mb")) + .withDescription("Maximum size of buffered actions per bulk request"); + + public static final ConfigOption BULK_FLUSH_INTERVAL_OPTION = + ConfigOptions.key("sink.bulk-flush.interval") + .durationType() + .defaultValue(Duration.ofSeconds(1)) + .withDescription("Bulk flush interval"); + + public static final ConfigOption BULK_FLUSH_BACKOFF_TYPE_OPTION = + ConfigOptions.key("sink.bulk-flush.backoff.strategy") + .enumType(BackOffType.class) + .defaultValue(BackOffType.DISABLED) + .withDescription("Backoff strategy"); + + public static final ConfigOption BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION = + ConfigOptions.key("sink.bulk-flush.backoff.max-retries") + .intType() + .noDefaultValue() + .withDescription("Maximum number of retries."); + + public static final ConfigOption BULK_FLUSH_BACKOFF_DELAY_OPTION = + ConfigOptions.key("sink.bulk-flush.backoff.delay") + .durationType() + .noDefaultValue() + .withDescription("Delay between each backoff attempt."); + + public static final ConfigOption CONNECTION_PATH_PREFIX = + ConfigOptions.key("connection.path-prefix") + .stringType() + .noDefaultValue() + .withDescription("Prefix string to be added to every REST communication."); + + public static final ConfigOption FORMAT_OPTION = + ConfigOptions.key("format") + .stringType() + .defaultValue("json") + .withDescription( + "The format must produce a valid JSON document. " + + "Please refer to the documentation on formats for more details."); + + // -------------------------------------------------------------------------------------------- + // Enums + // -------------------------------------------------------------------------------------------- + + /** + * Backoff strategy. Extends {@link ElasticsearchSinkBase.FlushBackoffType} with {@code + * DISABLED} option. + */ + public enum BackOffType { + DISABLED, + CONSTANT, + EXPONENTIAL + } + + private ElasticsearchConnectorOptions() {} +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java new file mode 100644 index 0000000000000..6452d00654884 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.types.logical.DistinctType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeFamily; +import org.apache.flink.table.types.logical.LogicalTypeRoot; + +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +/** Utility methods for validating Elasticsearch properties. */ +@Internal +class ElasticsearchValidationUtils { + + private static final Set ILLEGAL_PRIMARY_KEY_TYPES = new LinkedHashSet<>(); + + static { + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.ARRAY); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.MAP); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.MULTISET); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.STRUCTURED_TYPE); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.ROW); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.RAW); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BINARY); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.VARBINARY); + } + + /** + * Checks that the table does not have primary key defined on illegal types. In Elasticsearch + * the primary key is used to calculate the Elasticsearch document id, which is a string of up + * to 512 bytes. It cannot have whitespaces. As of now it is calculated by concatenating the + * fields. Certain types do not have a good string representation to be used in this scenario. + * The illegal types are mostly {@link LogicalTypeFamily#COLLECTION} types and {@link + * LogicalTypeRoot#RAW} type. + */ + public static void validatePrimaryKey(TableSchema schema) { + schema.getPrimaryKey() + .ifPresent( + key -> { + List illegalTypes = + key.getColumns().stream() + .map( + fieldName -> { + LogicalType logicalType = + schema.getFieldDataType(fieldName) + .get() + .getLogicalType(); + if (logicalType.is( + LogicalTypeRoot.DISTINCT_TYPE)) { + return ((DistinctType) logicalType) + .getSourceType() + .getTypeRoot(); + } else { + return logicalType.getTypeRoot(); + } + }) + .filter(ILLEGAL_PRIMARY_KEY_TYPES::contains) + .collect(Collectors.toList()); + + if (!illegalTypes.isEmpty()) { + throw new ValidationException( + String.format( + "The table has a primary key on columns of illegal types: %s.\n" + + " Elasticsearch sink does not support primary keys on columns of types: %s.", + illegalTypes, ILLEGAL_PRIMARY_KEY_TYPES)); + } + }); + } + + private ElasticsearchValidationUtils() {} +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java new file mode 100644 index 0000000000000..636f3409e7b92 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; + +import java.io.Serializable; + +/** This interface is responsible to generate index name from given {@link Row} record. */ +@Internal +interface IndexGenerator extends Serializable { + + /** + * Initialize the index generator, this will be called only once before {@link + * #generate(RowData)} is called. + */ + default void open() {} + + /** Generate index name according the the given row. */ + String generate(RowData row); +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.java new file mode 100644 index 0000000000000..adfcaa46d7425 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; + +import java.util.Objects; + +/** Base class for {@link IndexGenerator}. */ +@Internal +public abstract class IndexGeneratorBase implements IndexGenerator { + + private static final long serialVersionUID = 1L; + protected final String index; + + public IndexGeneratorBase(String index) { + this.index = index; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof IndexGeneratorBase)) { + return false; + } + IndexGeneratorBase that = (IndexGeneratorBase) o; + return index.equals(that.index); + } + + @Override + public int hashCode() { + return Objects.hash(index); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java new file mode 100644 index 0000000000000..8347a479e5a7a --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java @@ -0,0 +1,314 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; + +import javax.annotation.Nonnull; + +import java.io.Serializable; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Factory of {@link IndexGenerator}. + * + *

Flink supports both static index and dynamic index. + * + *

If you want to have a static index, this option value should be a plain string, e.g. + * 'myusers', all the records will be consistently written into "myusers" index. + * + *

If you want to have a dynamic index, you can use '{field_name}' to reference a field value in + * the record to dynamically generate a target index. You can also use + * '{field_name|date_format_string}' to convert a field value of TIMESTAMP/DATE/TIME type into the + * format specified by date_format_string. The date_format_string is compatible with {@link + * java.text.SimpleDateFormat}. For example, if the option value is 'myusers_{log_ts|yyyy-MM-dd}', + * then a record with log_ts field value 2020-03-27 12:25:55 will be written into + * "myusers_2020-03-27" index. + */ +@Internal +final class IndexGeneratorFactory { + + private IndexGeneratorFactory() {} + + public static IndexGenerator createIndexGenerator(String index, TableSchema schema) { + return createIndexGenerator(index, schema, ZoneId.systemDefault()); + } + + public static IndexGenerator createIndexGenerator( + String index, TableSchema schema, ZoneId localTimeZoneId) { + final IndexHelper indexHelper = new IndexHelper(); + if (indexHelper.checkIsDynamicIndex(index)) { + return createRuntimeIndexGenerator( + index, + schema.getFieldNames(), + schema.getFieldDataTypes(), + indexHelper, + localTimeZoneId); + } else { + return new StaticIndexGenerator(index); + } + } + + interface DynamicFormatter extends Serializable { + String format(@Nonnull Object fieldValue, DateTimeFormatter formatter); + } + + private static IndexGenerator createRuntimeIndexGenerator( + String index, + String[] fieldNames, + DataType[] fieldTypes, + IndexHelper indexHelper, + ZoneId localTimeZoneId) { + final String dynamicIndexPatternStr = indexHelper.extractDynamicIndexPatternStr(index); + final String indexPrefix = index.substring(0, index.indexOf(dynamicIndexPatternStr)); + final String indexSuffix = + index.substring(indexPrefix.length() + dynamicIndexPatternStr.length()); + + if (indexHelper.checkIsDynamicIndexWithSystemTimeFormat(index)) { + final String dateTimeFormat = + indexHelper.extractDateFormat( + index, LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE); + return new AbstractTimeIndexGenerator(index, dateTimeFormat) { + @Override + public String generate(RowData row) { + return indexPrefix + .concat(LocalDateTime.now(localTimeZoneId).format(dateTimeFormatter)) + .concat(indexSuffix); + } + }; + } + + final boolean isDynamicIndexWithFormat = indexHelper.checkIsDynamicIndexWithFormat(index); + final int indexFieldPos = + indexHelper.extractIndexFieldPos(index, fieldNames, isDynamicIndexWithFormat); + final LogicalType indexFieldType = fieldTypes[indexFieldPos].getLogicalType(); + final LogicalTypeRoot indexFieldLogicalTypeRoot = indexFieldType.getTypeRoot(); + + // validate index field type + indexHelper.validateIndexFieldType(indexFieldLogicalTypeRoot); + + // time extract dynamic index pattern + final RowData.FieldGetter fieldGetter = + RowData.createFieldGetter(indexFieldType, indexFieldPos); + + if (isDynamicIndexWithFormat) { + final String dateTimeFormat = + indexHelper.extractDateFormat(index, indexFieldLogicalTypeRoot); + DynamicFormatter formatFunction = + createFormatFunction( + indexFieldType, indexFieldLogicalTypeRoot, localTimeZoneId); + + return new AbstractTimeIndexGenerator(index, dateTimeFormat) { + @Override + public String generate(RowData row) { + Object fieldOrNull = fieldGetter.getFieldOrNull(row); + final String formattedField; + // TODO we can possibly optimize it to use the nullability of the field + if (fieldOrNull != null) { + formattedField = formatFunction.format(fieldOrNull, dateTimeFormatter); + } else { + formattedField = "null"; + } + return indexPrefix.concat(formattedField).concat(indexSuffix); + } + }; + } + // general dynamic index pattern + return new IndexGeneratorBase(index) { + @Override + public String generate(RowData row) { + Object indexField = fieldGetter.getFieldOrNull(row); + return indexPrefix + .concat(indexField == null ? "null" : indexField.toString()) + .concat(indexSuffix); + } + }; + } + + private static DynamicFormatter createFormatFunction( + LogicalType indexFieldType, + LogicalTypeRoot indexFieldLogicalTypeRoot, + ZoneId localTimeZoneId) { + switch (indexFieldLogicalTypeRoot) { + case DATE: + return (value, dateTimeFormatter) -> { + Integer indexField = (Integer) value; + return LocalDate.ofEpochDay(indexField).format(dateTimeFormatter); + }; + case TIME_WITHOUT_TIME_ZONE: + return (value, dateTimeFormatter) -> { + Integer indexField = (Integer) value; + return LocalTime.ofNanoOfDay(indexField * 1_000_000L).format(dateTimeFormatter); + }; + case TIMESTAMP_WITHOUT_TIME_ZONE: + return (value, dateTimeFormatter) -> { + TimestampData indexField = (TimestampData) value; + return indexField.toLocalDateTime().format(dateTimeFormatter); + }; + case TIMESTAMP_WITH_TIME_ZONE: + throw new UnsupportedOperationException( + "TIMESTAMP_WITH_TIME_ZONE is not supported yet"); + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return (value, dateTimeFormatter) -> { + TimestampData indexField = (TimestampData) value; + return indexField.toInstant().atZone(localTimeZoneId).format(dateTimeFormatter); + }; + default: + throw new TableException( + String.format( + "Unsupported type '%s' found in Elasticsearch dynamic index field, " + + "time-related pattern only support types are: DATE,TIME,TIMESTAMP.", + indexFieldType)); + } + } + + /** + * Helper class for {@link IndexGeneratorFactory}, this helper can use to validate index field + * type ans parse index format from pattern. + */ + static class IndexHelper { + private static final Pattern dynamicIndexPattern = Pattern.compile("\\{[^\\{\\}]+\\}?"); + private static final Pattern dynamicIndexTimeExtractPattern = + Pattern.compile(".*\\{.+\\|.*\\}.*"); + private static final Pattern dynamicIndexSystemTimeExtractPattern = + Pattern.compile( + ".*\\{\\s*(now\\(\\s*\\)|NOW\\(\\s*\\)|current_timestamp|CURRENT_TIMESTAMP)\\s*\\|.*\\}.*"); + private static final List supportedTypes = new ArrayList<>(); + private static final Map defaultFormats = new HashMap<>(); + + static { + // time related types + supportedTypes.add(LogicalTypeRoot.DATE); + supportedTypes.add(LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE); + supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE); + supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE); + supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE); + // general types + supportedTypes.add(LogicalTypeRoot.VARCHAR); + supportedTypes.add(LogicalTypeRoot.CHAR); + supportedTypes.add(LogicalTypeRoot.TINYINT); + supportedTypes.add(LogicalTypeRoot.INTEGER); + supportedTypes.add(LogicalTypeRoot.BIGINT); + } + + static { + defaultFormats.put(LogicalTypeRoot.DATE, "yyyy_MM_dd"); + defaultFormats.put(LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE, "HH_mm_ss"); + defaultFormats.put(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE, "yyyy_MM_dd_HH_mm_ss"); + defaultFormats.put(LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE, "yyyy_MM_dd_HH_mm_ss"); + defaultFormats.put( + LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE, "yyyy_MM_dd_HH_mm_ssX"); + } + + /** Validate the index field Type. */ + void validateIndexFieldType(LogicalTypeRoot logicalType) { + if (!supportedTypes.contains(logicalType)) { + throw new IllegalArgumentException( + String.format( + "Unsupported type %s of index field, " + "Supported types are: %s", + logicalType, supportedTypes)); + } + } + + /** Get the default date format. */ + String getDefaultFormat(LogicalTypeRoot logicalType) { + return defaultFormats.get(logicalType); + } + + /** Check general dynamic index is enabled or not by index pattern. */ + boolean checkIsDynamicIndex(String index) { + final Matcher matcher = dynamicIndexPattern.matcher(index); + int count = 0; + while (matcher.find()) { + count++; + } + if (count > 1) { + throw new TableException( + String.format( + "Chaining dynamic index pattern %s is not supported," + + " only support single dynamic index pattern.", + index)); + } + return count == 1; + } + + /** Check time extract dynamic index is enabled or not by index pattern. */ + boolean checkIsDynamicIndexWithFormat(String index) { + return dynamicIndexTimeExtractPattern.matcher(index).matches(); + } + + /** Check generate dynamic index is from system time or not. */ + boolean checkIsDynamicIndexWithSystemTimeFormat(String index) { + return dynamicIndexSystemTimeExtractPattern.matcher(index).matches(); + } + + /** Extract dynamic index pattern string from index pattern string. */ + String extractDynamicIndexPatternStr(String index) { + int start = index.indexOf("{"); + int end = index.lastIndexOf("}"); + return index.substring(start, end + 1); + } + + /** Extract index field position in a fieldNames, return the field position. */ + int extractIndexFieldPos( + String index, String[] fieldNames, boolean isDynamicIndexWithFormat) { + List fieldList = Arrays.asList(fieldNames); + String indexFieldName; + if (isDynamicIndexWithFormat) { + indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("|")); + } else { + indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("}")); + } + if (!fieldList.contains(indexFieldName)) { + throw new TableException( + String.format( + "Unknown field '%s' in index pattern '%s', please check the field name.", + indexFieldName, index)); + } + return fieldList.indexOf(indexFieldName); + } + + /** Extract dateTime format by the date format that extracted from index pattern string. */ + private String extractDateFormat(String index, LogicalTypeRoot logicalType) { + String format = index.substring(index.indexOf("|") + 1, index.indexOf("}")); + if ("".equals(format)) { + format = getDefaultFormat(logicalType); + } + return format; + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java new file mode 100644 index 0000000000000..ae7c522b4a9c5 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.TableColumn; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.DistinctType; +import org.apache.flink.table.types.logical.LogicalType; + +import java.io.Serializable; +import java.time.Duration; +import java.time.LocalDate; +import java.time.LocalTime; +import java.time.Period; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; + +/** An extractor for a Elasticsearch key from a {@link RowData}. */ +@Internal +class KeyExtractor implements Function, Serializable { + private final FieldFormatter[] fieldFormatters; + private final String keyDelimiter; + + private interface FieldFormatter extends Serializable { + String format(RowData rowData); + } + + private KeyExtractor(FieldFormatter[] fieldFormatters, String keyDelimiter) { + this.fieldFormatters = fieldFormatters; + this.keyDelimiter = keyDelimiter; + } + + @Override + public String apply(RowData rowData) { + final StringBuilder builder = new StringBuilder(); + for (int i = 0; i < fieldFormatters.length; i++) { + if (i > 0) { + builder.append(keyDelimiter); + } + final String value = fieldFormatters[i].format(rowData); + builder.append(value); + } + return builder.toString(); + } + + private static class ColumnWithIndex { + public TableColumn column; + public int index; + + public ColumnWithIndex(TableColumn column, int index) { + this.column = column; + this.index = index; + } + + public LogicalType getType() { + return column.getType().getLogicalType(); + } + + public int getIndex() { + return index; + } + } + + public static Function createKeyExtractor( + TableSchema schema, String keyDelimiter) { + return schema.getPrimaryKey() + .map( + key -> { + Map namesToColumns = new HashMap<>(); + List tableColumns = schema.getTableColumns(); + for (int i = 0; i < schema.getFieldCount(); i++) { + TableColumn column = tableColumns.get(i); + namesToColumns.put( + column.getName(), new ColumnWithIndex(column, i)); + } + + FieldFormatter[] fieldFormatters = + key.getColumns().stream() + .map(namesToColumns::get) + .map( + column -> + toFormatter( + column.index, column.getType())) + .toArray(FieldFormatter[]::new); + + return (Function) + new KeyExtractor(fieldFormatters, keyDelimiter); + }) + .orElseGet(() -> (Function & Serializable) (row) -> null); + } + + private static FieldFormatter toFormatter(int index, LogicalType type) { + switch (type.getTypeRoot()) { + case DATE: + return (row) -> LocalDate.ofEpochDay(row.getInt(index)).toString(); + case TIME_WITHOUT_TIME_ZONE: + return (row) -> + LocalTime.ofNanoOfDay((long) row.getInt(index) * 1_000_000L).toString(); + case INTERVAL_YEAR_MONTH: + return (row) -> Period.ofDays(row.getInt(index)).toString(); + case INTERVAL_DAY_TIME: + return (row) -> Duration.ofMillis(row.getLong(index)).toString(); + case DISTINCT_TYPE: + return toFormatter(index, ((DistinctType) type).getSourceType()); + default: + RowData.FieldGetter fieldGetter = RowData.createFieldGetter(type, index); + return (row) -> fieldGetter.getFieldOrNull(row).toString(); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java new file mode 100644 index 0000000000000..f5b241802b7b5 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.common.xcontent.XContentType; + +import java.io.Serializable; + +/** For version-agnostic creating of {@link ActionRequest}s. */ +@Internal +interface RequestFactory extends Serializable { + /** + * Creates an update request to be added to a {@link RequestIndexer}. Note: the type field has + * been deprecated since Elasticsearch 7.x and it would not take any effort. + */ + UpdateRequest createUpdateRequest( + String index, String docType, String key, XContentType contentType, byte[] document); + + /** + * Creates an index request to be added to a {@link RequestIndexer}. Note: the type field has + * been deprecated since Elasticsearch 7.x and it would not take any effort. + */ + IndexRequest createIndexRequest( + String index, String docType, String key, XContentType contentType, byte[] document); + + /** + * Creates a delete request to be added to a {@link RequestIndexer}. Note: the type field has + * been deprecated since Elasticsearch 7.x and it would not take any effort. + */ + DeleteRequest createDeleteRequest(String index, String docType, String key); +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java new file mode 100644 index 0000000000000..7fb1e4d4c1529 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.Preconditions; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.common.xcontent.XContentType; + +import javax.annotation.Nullable; + +import java.util.Objects; +import java.util.function.Function; + +/** Sink function for converting upserts into Elasticsearch {@link ActionRequest}s. */ +@Internal +class RowElasticsearchSinkFunction implements ElasticsearchSinkFunction { + + private static final long serialVersionUID = 1L; + + private final IndexGenerator indexGenerator; + private final String docType; + private final SerializationSchema serializationSchema; + private final XContentType contentType; + private final RequestFactory requestFactory; + private final Function createKey; + + public RowElasticsearchSinkFunction( + IndexGenerator indexGenerator, + @Nullable String docType, // this is deprecated in es 7+ + SerializationSchema serializationSchema, + XContentType contentType, + RequestFactory requestFactory, + Function createKey) { + this.indexGenerator = Preconditions.checkNotNull(indexGenerator); + this.docType = docType; + this.serializationSchema = Preconditions.checkNotNull(serializationSchema); + this.contentType = Preconditions.checkNotNull(contentType); + this.requestFactory = Preconditions.checkNotNull(requestFactory); + this.createKey = Preconditions.checkNotNull(createKey); + } + + @Override + public void open() { + indexGenerator.open(); + } + + @Override + public void process(RowData element, RuntimeContext ctx, RequestIndexer indexer) { + switch (element.getRowKind()) { + case INSERT: + case UPDATE_AFTER: + processUpsert(element, indexer); + break; + case UPDATE_BEFORE: + case DELETE: + processDelete(element, indexer); + break; + default: + throw new TableException("Unsupported message kind: " + element.getRowKind()); + } + } + + private void processUpsert(RowData row, RequestIndexer indexer) { + final byte[] document = serializationSchema.serialize(row); + final String key = createKey.apply(row); + if (key != null) { + final UpdateRequest updateRequest = + requestFactory.createUpdateRequest( + indexGenerator.generate(row), docType, key, contentType, document); + indexer.add(updateRequest); + } else { + final IndexRequest indexRequest = + requestFactory.createIndexRequest( + indexGenerator.generate(row), docType, key, contentType, document); + indexer.add(indexRequest); + } + } + + private void processDelete(RowData row, RequestIndexer indexer) { + final String key = createKey.apply(row); + final DeleteRequest deleteRequest = + requestFactory.createDeleteRequest(indexGenerator.generate(row), docType, key); + indexer.add(deleteRequest); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + RowElasticsearchSinkFunction that = (RowElasticsearchSinkFunction) o; + return Objects.equals(indexGenerator, that.indexGenerator) + && Objects.equals(docType, that.docType) + && Objects.equals(serializationSchema, that.serializationSchema) + && contentType == that.contentType + && Objects.equals(requestFactory, that.requestFactory) + && Objects.equals(createKey, that.createKey); + } + + @Override + public int hashCode() { + return Objects.hash( + indexGenerator, + docType, + serializationSchema, + contentType, + requestFactory, + createKey); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.java new file mode 100644 index 0000000000000..1ffcac489d21e --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.RowData; + +/** A static {@link IndexGenerator} which generate fixed index name. */ +@Internal +final class StaticIndexGenerator extends IndexGeneratorBase { + + public StaticIndexGenerator(String index) { + super(index); + } + + public String generate(RowData row) { + return index; + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java index 695f85c1d9220..4d1890fe78bcf 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java @@ -29,6 +29,7 @@ import java.util.stream.Stream; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -56,6 +57,12 @@ Stream testValidBuilders() { builder -> assertDoesNotThrow(builder::build)); } + @Test + void testDefaultDeliveryGuarantee() { + assertThat(createMinimalBuilder().build().getDeliveryGuarantee()) + .isEqualTo(DeliveryGuarantee.AT_LEAST_ONCE); + } + @Test void testThrowIfExactlyOnceConfigured() { assertThrows( diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java index d0132a0bb2817..3db12a400d09d 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java @@ -18,6 +18,7 @@ package org.apache.flink.connector.elasticsearch.sink; import org.apache.flink.api.common.operators.MailboxExecutor; +import org.apache.flink.api.connector.sink2.SinkWriter; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; import org.apache.flink.metrics.Counter; @@ -25,6 +26,7 @@ import org.apache.flink.metrics.groups.OperatorIOMetricGroup; import org.apache.flink.metrics.groups.SinkWriterMetricGroup; import org.apache.flink.metrics.testutils.MetricListener; +import org.apache.flink.runtime.metrics.MetricNames; import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.util.DockerImageVersions; @@ -38,8 +40,11 @@ import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestHighLevelClient; @@ -58,8 +63,11 @@ import java.io.IOException; import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import java.util.Optional; +import static org.apache.flink.connector.elasticsearch.sink.TestClientBase.DOCUMENT_TYPE; import static org.apache.flink.connector.elasticsearch.sink.TestClientBase.buildMessage; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.greaterThan; @@ -204,6 +212,30 @@ void testIncrementByteOutMetric() throws Exception { } } + @Test + void testIncrementRecordsSendMetric() throws Exception { + final String index = "test-inc-records-send"; + final int flushAfterNActions = 2; + final BulkProcessorConfig bulkProcessorConfig = + new BulkProcessorConfig(flushAfterNActions, -1, -1, FlushBackoffType.NONE, 0, 0); + + try (final ElasticsearchWriter> writer = + createWriter(index, false, bulkProcessorConfig)) { + final Optional recordsSend = + metricListener.getCounter(MetricNames.NUM_RECORDS_SEND); + writer.write(Tuple2.of(1, buildMessage(1)), null); + // Update existing index + writer.write(Tuple2.of(1, "u" + buildMessage(2)), null); + // Delete index + writer.write(Tuple2.of(1, "d" + buildMessage(3)), null); + + writer.blockingFlushAllActions(); + + assertTrue(recordsSend.isPresent()); + assertEquals(recordsSend.get().getCount(), 3L); + } + } + @Test void testCurrentSendTime() throws Exception { final String index = "test-current-send-time"; @@ -239,9 +271,9 @@ private ElasticsearchWriter> createWriter( boolean flushOnCheckpoint, BulkProcessorConfig bulkProcessorConfig, SinkWriterMetricGroup metricGroup) { - return new ElasticsearchWriter>( + return new ElasticsearchWriter<>( Collections.singletonList(HttpHost.create(ES_CONTAINER.getHttpHostAddress())), - TestEmitter.jsonEmitter(index, context.getDataFieldName()), + new UpdatingEmitter(index, context.getDataFieldName()), flushOnCheckpoint, bulkProcessorConfig, new TestBulkProcessorBuilderFactory(), @@ -310,6 +342,50 @@ public void accept( } } + private static class UpdatingEmitter implements ElasticsearchEmitter> { + + private final String dataFieldName; + private final String index; + + UpdatingEmitter(String index, String dataFieldName) { + this.index = index; + this.dataFieldName = dataFieldName; + } + + @Override + public void emit( + Tuple2 element, + SinkWriter.Context context, + RequestIndexer indexer) { + + Map document = new HashMap<>(); + document.put(dataFieldName, element.f1); + + final char action = element.f1.charAt(0); + final String id = element.f0.toString(); + switch (action) { + case 'd': + { + indexer.add(new DeleteRequest(index).id(id)); + break; + } + case 'u': + { + indexer.add(new UpdateRequest().index(index).id(id).doc(document)); + break; + } + default: + { + indexer.add( + new IndexRequest(index) + .id(id) + .type(DOCUMENT_TYPE) + .source(document)); + } + } + } + } + private static class TestClient extends TestClientBase { TestClient(RestHighLevelClient client) { diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java index 49943b23bedd0..2cfc5eb4bcc1d 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java @@ -58,7 +58,7 @@ public void emit( indexer.add(createIndexRequest(element)); } - public IndexRequest createIndexRequest(Tuple2 element) { + private IndexRequest createIndexRequest(Tuple2 element) { Map document = new HashMap<>(); document.put(dataFieldName, element.f1); try { diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java index 21ad5fd2eac6f..75dd9b98d9b23 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java @@ -25,6 +25,7 @@ import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.UniqueConstraint; @@ -310,10 +311,7 @@ public void testWritingDocumentsWithDynamicIndexFromSystemTime() throws Exceptio TableEnvironment.create(EnvironmentSettings.inStreamingMode()); DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd"); - tableEnvironment - .getConfig() - .getConfiguration() - .setString("table.local-time-zone", "Asia/Shanghai"); + tableEnvironment.getConfig().set(TableConfigOptions.LOCAL_TIME_ZONE, "Asia/Shanghai"); String dynamicIndex1 = "dynamic-index-" diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorTest.java index 7840bfa81a85a..8760d0d4a4372 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorTest.java @@ -35,11 +35,14 @@ import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; +import java.time.ZoneId; import java.time.format.DateTimeFormatter; import java.time.temporal.UnsupportedTemporalTypeException; import java.util.Arrays; import java.util.List; +import static org.junit.jupiter.api.Assumptions.assumingThat; + /** Suite tests for {@link IndexGenerator}. */ public class IndexGeneratorTest { @@ -54,6 +57,7 @@ public class IndexGeneratorTest { "local_datetime", "local_date", "local_time", + "local_timestamp", "note", "status"); @@ -68,6 +72,7 @@ public class IndexGeneratorTest { DataTypes.TIMESTAMP().bridgedTo(LocalDateTime.class), DataTypes.DATE().bridgedTo(LocalDate.class), DataTypes.TIME().bridgedTo(LocalTime.class), + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(), DataTypes.STRING(), DataTypes.BOOLEAN()); @@ -86,6 +91,10 @@ public class IndexGeneratorTest { LocalDateTime.of(2020, 3, 18, 12, 12, 14, 1000)), (int) LocalDate.of(2020, 3, 18).toEpochDay(), (int) (LocalTime.of(12, 13, 14, 2000).toNanoOfDay() / 1_000_000L), + TimestampData.fromInstant( + LocalDateTime.of(2020, 3, 18, 3, 12, 14, 1000) + .atZone(ZoneId.of("Asia/Shanghai")) + .toInstant()), "test1", true), GenericRowData.of( @@ -101,9 +110,44 @@ public class IndexGeneratorTest { LocalDateTime.of(2020, 3, 19, 12, 22, 14, 1000)), (int) LocalDate.of(2020, 3, 19).toEpochDay(), (int) (LocalTime.of(12, 13, 14, 2000).toNanoOfDay() / 1_000_000L), + TimestampData.fromInstant( + LocalDateTime.of(2020, 3, 19, 20, 22, 14, 1000) + .atZone(ZoneId.of("America/Los_Angeles")) + .toInstant()), "test2", false)); + @Test + public void testDynamicIndexFromTimestampTzUTC() { + assumingThat( + ZoneId.systemDefault().equals(ZoneId.of("UTC")), + () -> { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "{local_timestamp|yyyy_MM_dd_HH-ss}_index", + fieldNames, + dataTypes); + indexGenerator.open(); + Assertions.assertEquals( + "2020_03_17_19-14_index", indexGenerator.generate(rows.get(0))); + Assertions.assertEquals( + "2020_03_20_03-14_index", indexGenerator.generate(rows.get(1))); + }); + } + + @Test + public void testDynamicIndexFromTimestampTzWithSpecificTimezone() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "{local_timestamp|yyyy_MM_dd_HH-ss}_index", + fieldNames, + dataTypes, + ZoneId.of("Europe/Berlin")); + indexGenerator.open(); + Assertions.assertEquals("2020_03_17_20-14_index", indexGenerator.generate(rows.get(0))); + Assertions.assertEquals("2020_03_20_04-14_index", indexGenerator.generate(rows.get(1))); + } + @Test public void testDynamicIndexFromTimestamp() { IndexGenerator indexGenerator = diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java new file mode 100644 index 0000000000000..bbb5081b73389 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java @@ -0,0 +1,305 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.util.TestLogger; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.sql.Timestamp; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; +import java.time.temporal.UnsupportedTemporalTypeException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.hamcrest.CoreMatchers.is; +import static org.junit.Assume.assumeThat; + +/** Tests for {@link IndexGeneratorFactory}. */ +public class IndexGeneratorFactoryTest extends TestLogger { + + private TableSchema schema; + private List rows; + + @Before + public void prepareData() { + schema = + new TableSchema.Builder() + .field("id", DataTypes.INT()) + .field("item", DataTypes.STRING()) + .field("log_ts", DataTypes.BIGINT()) + .field("log_date", DataTypes.DATE()) + .field("log_time", DataTypes.TIME()) + .field("order_timestamp", DataTypes.TIMESTAMP()) + .field("local_timestamp", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()) + .field("status", DataTypes.BOOLEAN()) + .build(); + + rows = new ArrayList<>(); + rows.add( + GenericRowData.of( + 1, + StringData.fromString("apple"), + Timestamp.valueOf("2020-03-18 12:12:14").getTime(), + (int) LocalDate.parse("2020-03-18").toEpochDay(), + (int) (LocalTime.parse("12:12:14").toNanoOfDay() / 1_000_000L), + TimestampData.fromLocalDateTime(LocalDateTime.parse("2020-03-18T12:12:14")), + TimestampData.fromInstant( + LocalDateTime.of(2020, 3, 18, 3, 12, 14, 1000) + .atZone(ZoneId.of("Asia/Shanghai")) + .toInstant()), + true)); + rows.add( + GenericRowData.of( + 2, + StringData.fromString("peanut"), + Timestamp.valueOf("2020-03-19 12:12:14").getTime(), + (int) LocalDate.parse("2020-03-19").toEpochDay(), + (int) (LocalTime.parse("12:22:21").toNanoOfDay() / 1_000_000L), + TimestampData.fromLocalDateTime(LocalDateTime.parse("2020-03-19T12:22:14")), + TimestampData.fromInstant( + LocalDateTime.of(2020, 3, 19, 20, 22, 14, 1000) + .atZone(ZoneId.of("America/Los_Angeles")) + .toInstant()), + false)); + } + + @Test + public void testDynamicIndexFromTimestamp() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "{order_timestamp|yyyy_MM_dd_HH-ss}_index", schema); + indexGenerator.open(); + Assert.assertEquals("2020_03_18_12-14_index", indexGenerator.generate(rows.get(0))); + IndexGenerator indexGenerator1 = + IndexGeneratorFactory.createIndexGenerator( + "{order_timestamp|yyyy_MM_dd_HH_mm}_index", schema); + indexGenerator1.open(); + Assert.assertEquals("2020_03_19_12_22_index", indexGenerator1.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromDate() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "my-index-{log_date|yyyy/MM/dd}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-2020/03/18", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-2020/03/19", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromTime() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("my-index-{log_time|HH-mm}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-12-12", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-12-22", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexDefaultFormat() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("my-index-{log_time|}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-12_12_14", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-12_22_21", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromSystemTime() { + List supportedUseCases = + Arrays.asList( + "now()", + "NOW()", + "now( )", + "NOW(\t)", + "\t NOW( ) \t", + "current_timestamp", + "CURRENT_TIMESTAMP", + "\tcurrent_timestamp\t", + " current_timestamp "); + + supportedUseCases.stream() + .forEach( + f -> { + DateTimeFormatter dateTimeFormatter = + DateTimeFormatter.ofPattern("yyyy_MM_dd"); + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + String.format("my-index-{%s|yyyy_MM_dd}", f), schema); + indexGenerator.open(); + // The date may change during the running of the unit test. + // Generate expected index-name based on the current time + // before and after calling the generate method. + String expectedIndex1 = + "my-index-" + LocalDateTime.now().format(dateTimeFormatter); + String actualIndex = indexGenerator.generate(rows.get(1)); + String expectedIndex2 = + "my-index-" + LocalDateTime.now().format(dateTimeFormatter); + Assert.assertTrue( + actualIndex.equals(expectedIndex1) + || actualIndex.equals(expectedIndex2)); + }); + + List invalidUseCases = + Arrays.asList( + "now", + "now(", + "NOW", + "NOW)", + "current_timestamp()", + "CURRENT_TIMESTAMP()", + "CURRENT_timestamp"); + invalidUseCases.stream() + .forEach( + f -> { + String expectedExceptionMsg = + String.format( + "Unknown field '%s' in index pattern 'my-index-{%s|yyyy_MM_dd}'," + + " please check the field name.", + f, f); + try { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + String.format("my-index-{%s|yyyy_MM_dd}", f), + schema); + indexGenerator.open(); + } catch (TableException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + }); + } + + @Test + public void testDynamicIndexDefaultFormatTimestampWithLocalTimeZoneUTC() { + assumeThat(ZoneId.systemDefault(), is(ZoneId.of("UTC"))); + + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("my-index-{local_timestamp|}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-2020_03_17_19_12_14Z", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-2020_03_20_03_22_14Z", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexDefaultFormatTimestampWithLocalTimeZoneWithSpecificTimeZone() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_timestamp|}", schema, ZoneId.of("Europe/Berlin")); + indexGenerator.open(); + Assert.assertEquals( + "my-index-2020_03_17_20_12_14+01", indexGenerator.generate(rows.get(0))); + Assert.assertEquals( + "my-index-2020_03_20_04_22_14+01", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testGeneralDynamicIndex() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("index_{item}", schema); + indexGenerator.open(); + Assert.assertEquals("index_apple", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("index_peanut", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testStaticIndex() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("my-index", schema); + indexGenerator.open(); + Assert.assertEquals("my-index", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testUnknownField() { + String expectedExceptionMsg = + "Unknown field 'unknown_ts' in index pattern 'my-index-{unknown_ts|yyyy-MM-dd}'," + + " please check the field name."; + try { + IndexGeneratorFactory.createIndexGenerator("my-index-{unknown_ts|yyyy-MM-dd}", schema); + } catch (TableException e) { + Assert.assertEquals(e.getMessage(), expectedExceptionMsg); + } + } + + @Test + public void testUnsupportedTimeType() { + String expectedExceptionMsg = + "Unsupported type 'INT' found in Elasticsearch dynamic index field, " + + "time-related pattern only support types are: DATE,TIME,TIMESTAMP."; + try { + IndexGeneratorFactory.createIndexGenerator("my-index-{id|yyyy-MM-dd}", schema); + } catch (TableException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testUnsupportedMultiParametersType() { + String expectedExceptionMsg = + "Chaining dynamic index pattern my-index-{local_date}-{local_time} is not supported," + + " only support single dynamic index pattern."; + try { + IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_date}-{local_time}", schema); + } catch (TableException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testDynamicIndexUnsupportedFormat() { + String expectedExceptionMsg = "Unsupported field: HourOfDay"; + try { + IndexGeneratorFactory.createIndexGenerator( + "my-index-{log_date|yyyy/MM/dd HH:mm}", schema); + } catch (UnsupportedTemporalTypeException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testUnsupportedIndexFieldType() { + String expectedExceptionMsg = + "Unsupported type BOOLEAN of index field, Supported types are:" + + " [DATE, TIME_WITHOUT_TIME_ZONE, TIMESTAMP_WITHOUT_TIME_ZONE, TIMESTAMP_WITH_TIME_ZONE," + + " TIMESTAMP_WITH_LOCAL_TIME_ZONE, VARCHAR, CHAR, TINYINT, INTEGER, BIGINT]"; + try { + IndexGeneratorFactory.createIndexGenerator("index_{status}", schema); + } catch (IllegalArgumentException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java new file mode 100644 index 0000000000000..bcfb68da0b710 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; + +import org.junit.Test; + +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.function.Function; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.junit.Assert.assertThat; + +/** Tests for {@link KeyExtractor}. */ +public class KeyExtractorTest { + @Test + public void testSimpleKey() { + TableSchema schema = + TableSchema.builder() + .field("a", DataTypes.BIGINT().notNull()) + .field("b", DataTypes.STRING()) + .primaryKey("a") + .build(); + + Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); + + String key = keyExtractor.apply(GenericRowData.of(12L, StringData.fromString("ABCD"))); + assertThat(key, equalTo("12")); + } + + @Test + public void testNoPrimaryKey() { + TableSchema schema = + TableSchema.builder() + .field("a", DataTypes.BIGINT().notNull()) + .field("b", DataTypes.STRING()) + .build(); + + Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); + + String key = keyExtractor.apply(GenericRowData.of(12L, StringData.fromString("ABCD"))); + assertThat(key, nullValue()); + } + + @Test + public void testTwoFieldsKey() { + TableSchema schema = + TableSchema.builder() + .field("a", DataTypes.BIGINT().notNull()) + .field("b", DataTypes.STRING()) + .field("c", DataTypes.TIMESTAMP().notNull()) + .primaryKey("a", "c") + .build(); + + Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); + + String key = + keyExtractor.apply( + GenericRowData.of( + 12L, + StringData.fromString("ABCD"), + TimestampData.fromLocalDateTime( + LocalDateTime.parse("2012-12-12T12:12:12")))); + assertThat(key, equalTo("12_2012-12-12T12:12:12")); + } + + @Test + public void testAllTypesKey() { + TableSchema schema = + TableSchema.builder() + .field("a", DataTypes.TINYINT().notNull()) + .field("b", DataTypes.SMALLINT().notNull()) + .field("c", DataTypes.INT().notNull()) + .field("d", DataTypes.BIGINT().notNull()) + .field("e", DataTypes.BOOLEAN().notNull()) + .field("f", DataTypes.FLOAT().notNull()) + .field("g", DataTypes.DOUBLE().notNull()) + .field("h", DataTypes.STRING().notNull()) + .field("i", DataTypes.TIMESTAMP().notNull()) + .field("j", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE().notNull()) + .field("k", DataTypes.TIME().notNull()) + .field("l", DataTypes.DATE().notNull()) + .primaryKey("a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l") + .build(); + + Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); + + String key = + keyExtractor.apply( + GenericRowData.of( + (byte) 1, + (short) 2, + 3, + (long) 4, + true, + 1.0f, + 2.0d, + StringData.fromString("ABCD"), + TimestampData.fromLocalDateTime( + LocalDateTime.parse("2012-12-12T12:12:12")), + TimestampData.fromInstant(Instant.parse("2013-01-13T13:13:13Z")), + (int) (LocalTime.parse("14:14:14").toNanoOfDay() / 1_000_000), + (int) LocalDate.parse("2015-05-15").toEpochDay())); + assertThat( + key, + equalTo( + "1_2_3_4_true_1.0_2.0_ABCD_2012-12-12T12:12:12_2013-01-13T13:13:13_14:14:14_2015-05-15")); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java new file mode 100644 index 0000000000000..40405142fe3b1 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.factories.FactoryUtil; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +/** A utility class for mocking {@link DynamicTableFactory.Context}. */ +class TestContext { + + private ResolvedSchema schema = ResolvedSchema.of(Column.physical("a", DataTypes.TIME())); + + private final Map options = new HashMap<>(); + + public static TestContext context() { + return new TestContext(); + } + + public TestContext withSchema(ResolvedSchema schema) { + this.schema = schema; + return this; + } + + DynamicTableFactory.Context build() { + return new FactoryUtil.DefaultDynamicTableContext( + ObjectIdentifier.of("default", "default", "t1"), + new ResolvedCatalogTable( + CatalogTable.of( + Schema.newBuilder().fromResolvedSchema(schema).build(), + "mock context", + Collections.emptyList(), + options), + schema), + Collections.emptyMap(), + new Configuration(), + TestContext.class.getClassLoader(), + false); + } + + public TestContext withOption(String key, String value) { + options.put(key, value); + return this; + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 92bc8f125e824..df955cb72552f 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. @@ -199,33 +199,6 @@ under the License. 1 - - org.apache.maven.plugins - maven-shade-plugin - - - shade-flink - package - - shade - - - - - org.apache.flink:flink-connector-base - org.apache.flink:flink-connector-elasticsearch-base - - - - - org.apache.flink.connector.base - org.apache.flink.connector.elasticsearch6.shaded.org.apache.flink.connector.base - - - - - - diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java index 34c098b85aed3..c90ccaca11c68 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java @@ -46,7 +46,6 @@ * .source(element.f1) * ); * }) - * .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE) * .build(); * } * diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java new file mode 100644 index 0000000000000..8b8332167c8d7 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.ValidationException; + +import org.apache.http.HttpHost; + +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; + +/** Elasticsearch 6 specific configuration. */ +@Internal +final class Elasticsearch6Configuration extends ElasticsearchConfiguration { + Elasticsearch6Configuration(ReadableConfig config, ClassLoader classLoader) { + super(config, classLoader); + } + + public List getHosts() { + return config.get(HOSTS_OPTION).stream() + .map(Elasticsearch6Configuration::validateAndParseHostsString) + .collect(Collectors.toList()); + } + + /** + * Parse Hosts String to list. + * + *

Hosts String format was given as following: + * + *

+     *     connector.hosts = http://host_name:9092;http://host_name:9093
+     * 
+ */ + private static HttpHost validateAndParseHostsString(String host) { + try { + HttpHost httpHost = HttpHost.create(host); + if (httpHost.getPort() < 0) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing port.", + host, HOSTS_OPTION.key())); + } + + if (httpHost.getSchemeName() == null) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing scheme.", + host, HOSTS_OPTION.key())); + } + return httpHost; + } catch (Exception e) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'.", + host, HOSTS_OPTION.key()), + e); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java new file mode 100644 index 0000000000000..0c7c756d890b3 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java @@ -0,0 +1,335 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSink; +import org.apache.flink.streaming.connectors.elasticsearch6.RestClientFactory; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkFunctionProvider; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.StringUtils; + +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.common.xcontent.XContentType; + +import javax.annotation.Nullable; + +import java.time.ZoneId; +import java.util.List; +import java.util.Objects; + +/** + * A {@link DynamicTableSink} that describes how to create a {@link ElasticsearchSink} from a + * logical description. + */ +@PublicEvolving +final class Elasticsearch6DynamicSink implements DynamicTableSink { + @VisibleForTesting + static final Elasticsearch6RequestFactory REQUEST_FACTORY = new Elasticsearch6RequestFactory(); + + private final EncodingFormat> format; + private final TableSchema schema; + private final Elasticsearch6Configuration config; + private final ZoneId localTimeZoneId; + private final boolean isDynamicIndexWithSystemTime; + + public Elasticsearch6DynamicSink( + EncodingFormat> format, + Elasticsearch6Configuration config, + TableSchema schema, + ZoneId localTimeZoneId) { + this(format, config, schema, localTimeZoneId, (ElasticsearchSink.Builder::new)); + } + + // -------------------------------------------------------------- + // Hack to make configuration testing possible. + // + // The code in this block should never be used outside of tests. + // Having a way to inject a builder we can assert the builder in + // the test. We can not assert everything though, e.g. it is not + // possible to assert flushing on checkpoint, as it is configured + // on the sink itself. + // -------------------------------------------------------------- + + private final ElasticSearchBuilderProvider builderProvider; + + @FunctionalInterface + interface ElasticSearchBuilderProvider { + ElasticsearchSink.Builder createBuilder( + List httpHosts, RowElasticsearchSinkFunction upsertSinkFunction); + } + + Elasticsearch6DynamicSink( + EncodingFormat> format, + Elasticsearch6Configuration config, + TableSchema schema, + ZoneId localTimeZoneId, + ElasticSearchBuilderProvider builderProvider) { + this.format = format; + this.schema = schema; + this.config = config; + this.localTimeZoneId = localTimeZoneId; + this.isDynamicIndexWithSystemTime = isDynamicIndexWithSystemTime(); + this.builderProvider = builderProvider; + } + + // -------------------------------------------------------------- + // End of hack to make configuration testing possible + // -------------------------------------------------------------- + + public boolean isDynamicIndexWithSystemTime() { + IndexGeneratorFactory.IndexHelper indexHelper = new IndexGeneratorFactory.IndexHelper(); + return indexHelper.checkIsDynamicIndexWithSystemTimeFormat(config.getIndex()); + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { + ChangelogMode.Builder builder = ChangelogMode.newBuilder(); + for (RowKind kind : requestedMode.getContainedKinds()) { + if (kind != RowKind.UPDATE_BEFORE) { + builder.addContainedKind(kind); + } + } + if (isDynamicIndexWithSystemTime && !requestedMode.containsOnly(RowKind.INSERT)) { + throw new ValidationException( + "Dynamic indexing based on system time only works on append only stream."); + } + return builder.build(); + } + + @Override + public SinkFunctionProvider getSinkRuntimeProvider(Context context) { + return () -> { + SerializationSchema format = + this.format.createRuntimeEncoder(context, schema.toRowDataType()); + + final RowElasticsearchSinkFunction upsertFunction = + new RowElasticsearchSinkFunction( + IndexGeneratorFactory.createIndexGenerator( + config.getIndex(), schema, localTimeZoneId), + config.getDocumentType(), + format, + XContentType.JSON, + REQUEST_FACTORY, + KeyExtractor.createKeyExtractor(schema, config.getKeyDelimiter())); + + final ElasticsearchSink.Builder builder = + builderProvider.createBuilder(config.getHosts(), upsertFunction); + + builder.setFailureHandler(config.getFailureHandler()); + builder.setBulkFlushMaxActions(config.getBulkFlushMaxActions()); + builder.setBulkFlushMaxSizeMb((int) (config.getBulkFlushMaxByteSize() >> 20)); + builder.setBulkFlushInterval(config.getBulkFlushInterval()); + builder.setBulkFlushBackoff(config.isBulkFlushBackoffEnabled()); + config.getBulkFlushBackoffType().ifPresent(builder::setBulkFlushBackoffType); + config.getBulkFlushBackoffRetries().ifPresent(builder::setBulkFlushBackoffRetries); + config.getBulkFlushBackoffDelay().ifPresent(builder::setBulkFlushBackoffDelay); + + // we must overwrite the default factory which is defined with a lambda because of a bug + // in shading lambda serialization shading see FLINK-18006 + if (config.getUsername().isPresent() + && config.getPassword().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get()) + && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get())) { + builder.setRestClientFactory( + new AuthRestClientFactory( + config.getPathPrefix().orElse(null), + config.getUsername().get(), + config.getPassword().get())); + } else { + builder.setRestClientFactory( + new DefaultRestClientFactory(config.getPathPrefix().orElse(null))); + } + + final ElasticsearchSink sink = builder.build(); + + if (config.isDisableFlushOnCheckpoint()) { + sink.disableFlushOnCheckpoint(); + } + + return sink; + }; + } + + @Override + public DynamicTableSink copy() { + return this; + } + + @Override + public String asSummaryString() { + return "Elasticsearch6"; + } + + /** Serializable {@link RestClientFactory} used by the sink. */ + @VisibleForTesting + static class DefaultRestClientFactory implements RestClientFactory { + + private final String pathPrefix; + + public DefaultRestClientFactory(@Nullable String pathPrefix) { + this.pathPrefix = pathPrefix; + } + + @Override + public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { + if (pathPrefix != null) { + restClientBuilder.setPathPrefix(pathPrefix); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DefaultRestClientFactory that = (DefaultRestClientFactory) o; + return Objects.equals(pathPrefix, that.pathPrefix); + } + + @Override + public int hashCode() { + return Objects.hash(pathPrefix); + } + } + + /** Serializable {@link RestClientFactory} used by the sink which enable authentication. */ + @VisibleForTesting + static class AuthRestClientFactory implements RestClientFactory { + + private final String pathPrefix; + private final String username; + private final String password; + private transient CredentialsProvider credentialsProvider; + + public AuthRestClientFactory( + @Nullable String pathPrefix, String username, String password) { + this.pathPrefix = pathPrefix; + this.password = password; + this.username = username; + } + + @Override + public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { + if (pathPrefix != null) { + restClientBuilder.setPathPrefix(pathPrefix); + } + if (credentialsProvider == null) { + credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials( + AuthScope.ANY, new UsernamePasswordCredentials(username, password)); + } + restClientBuilder.setHttpClientConfigCallback( + httpAsyncClientBuilder -> + httpAsyncClientBuilder.setDefaultCredentialsProvider( + credentialsProvider)); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + AuthRestClientFactory that = (AuthRestClientFactory) o; + return Objects.equals(pathPrefix, that.pathPrefix) + && Objects.equals(username, that.username) + && Objects.equals(password, that.password); + } + + @Override + public int hashCode() { + return Objects.hash(pathPrefix, username, password); + } + } + + /** + * Version-specific creation of {@link org.elasticsearch.action.ActionRequest}s used by the + * sink. + */ + private static class Elasticsearch6RequestFactory implements RequestFactory { + @Override + public UpdateRequest createUpdateRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document) { + return new UpdateRequest(index, docType, key) + .doc(document, contentType) + .upsert(document, contentType); + } + + @Override + public IndexRequest createIndexRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document) { + return new IndexRequest(index, docType, key).source(document, contentType); + } + + @Override + public DeleteRequest createDeleteRequest(String index, String docType, String key) { + return new DeleteRequest(index, docType, key); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Elasticsearch6DynamicSink that = (Elasticsearch6DynamicSink) o; + return Objects.equals(format, that.format) + && Objects.equals(schema, that.schema) + && Objects.equals(config, that.config) + && Objects.equals(builderProvider, that.builderProvider); + } + + @Override + public int hashCode() { + return Objects.hash(format, schema, config, builderProvider); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java new file mode 100644 index 0000000000000..73f5cf889de43 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.utils.TableSchemaUtils; +import org.apache.flink.util.StringUtils; + +import java.time.ZoneId; +import java.util.Set; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FORMAT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.INDEX_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; + +/** A {@link DynamicTableSinkFactory} for discovering {@link Elasticsearch6DynamicSink}. */ +@Internal +public class Elasticsearch6DynamicSinkFactory implements DynamicTableSinkFactory { + private static final Set> requiredOptions = + Stream.of(HOSTS_OPTION, INDEX_OPTION, DOCUMENT_TYPE_OPTION).collect(Collectors.toSet()); + private static final Set> optionalOptions = + Stream.of( + KEY_DELIMITER_OPTION, + FAILURE_HANDLER_OPTION, + FLUSH_ON_CHECKPOINT_OPTION, + BULK_FLASH_MAX_SIZE_OPTION, + BULK_FLUSH_MAX_ACTIONS_OPTION, + BULK_FLUSH_INTERVAL_OPTION, + BULK_FLUSH_BACKOFF_TYPE_OPTION, + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION, + BULK_FLUSH_BACKOFF_DELAY_OPTION, + CONNECTION_PATH_PREFIX, + FORMAT_OPTION, + PASSWORD_OPTION, + USERNAME_OPTION) + .collect(Collectors.toSet()); + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + TableSchema tableSchema = context.getCatalogTable().getSchema(); + ElasticsearchValidationUtils.validatePrimaryKey(tableSchema); + final FactoryUtil.TableFactoryHelper helper = + FactoryUtil.createTableFactoryHelper(this, context); + + final EncodingFormat> format = + helper.discoverEncodingFormat(SerializationFormatFactory.class, FORMAT_OPTION); + + helper.validate(); + Configuration configuration = new Configuration(); + context.getCatalogTable().getOptions().forEach(configuration::setString); + Elasticsearch6Configuration config = + new Elasticsearch6Configuration(configuration, context.getClassLoader()); + + validate(config, configuration); + + return new Elasticsearch6DynamicSink( + format, + config, + TableSchemaUtils.getPhysicalSchema(tableSchema), + getLocalTimeZoneId(context.getConfiguration())); + } + + ZoneId getLocalTimeZoneId(ReadableConfig readableConfig) { + final String zone = readableConfig.get(TableConfigOptions.LOCAL_TIME_ZONE); + final ZoneId zoneId = + TableConfigOptions.LOCAL_TIME_ZONE.defaultValue().equals(zone) + ? ZoneId.systemDefault() + : ZoneId.of(zone); + + return zoneId; + } + + private void validate(Elasticsearch6Configuration config, Configuration originalConfiguration) { + config.getFailureHandler(); // checks if we can instantiate the custom failure handler + config.getHosts(); // validate hosts + validate( + config.getIndex().length() >= 1, + () -> String.format("'%s' must not be empty", INDEX_OPTION.key())); + int maxActions = config.getBulkFlushMaxActions(); + validate( + maxActions == -1 || maxActions >= 1, + () -> + String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_MAX_ACTIONS_OPTION.key(), maxActions)); + long maxSize = config.getBulkFlushMaxByteSize(); + long mb1 = 1024 * 1024; + validate( + maxSize == -1 || (maxSize >= mb1 && maxSize % mb1 == 0), + () -> + String.format( + "'%s' must be in MB granularity. Got: %s", + BULK_FLASH_MAX_SIZE_OPTION.key(), + originalConfiguration + .get(BULK_FLASH_MAX_SIZE_OPTION) + .toHumanReadableString())); + validate( + config.getBulkFlushBackoffRetries().map(retries -> retries >= 1).orElse(true), + () -> + String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), + config.getBulkFlushBackoffRetries().get())); + if (config.getUsername().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) { + validate( + config.getPassword().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get()), + () -> + String.format( + "'%s' and '%s' must be set at the same time. Got: username '%s' and password '%s'", + USERNAME_OPTION.key(), + PASSWORD_OPTION.key(), + config.getUsername().get(), + config.getPassword().orElse(""))); + } + } + + private static void validate(boolean condition, Supplier message) { + if (!condition) { + throw new ValidationException(message.get()); + } + } + + @Override + public String factoryIdentifier() { + return "elasticsearch-6"; + } + + @Override + public Set> requiredOptions() { + return requiredOptions; + } + + @Override + public Set> optionalOptions() { + return optionalOptions; + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connectors/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory index bb5a894bd5b27..29a859380f7ee 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.connector.elasticsearch.table.Elasticsearch6DynamicSinkFactory +org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch6DynamicSinkFactory diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java new file mode 100644 index 0000000000000..e99abbe90410e --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java @@ -0,0 +1,250 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.api.common.typeutils.base.VoidSerializer; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.util.TestLogger; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.Arrays; +import java.util.Collections; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; + +/** Tests for validation in {@link Elasticsearch6DynamicSinkFactory}. */ +public class Elasticsearch6DynamicSinkFactoryTest extends TestLogger { + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Test + public void validateEmptyConfiguration() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "One or more required options are missing.\n" + + "\n" + + "Missing required options are:\n" + + "\n" + + "document-type\n" + + "hosts\n" + + "index"); + sinkFactory.createDynamicTableSink(context().build()); + } + + @Test + public void validateWrongIndex() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("'index' must not be empty"); + sinkFactory.createDynamicTableSink( + context() + .withOption("index", "") + .withOption("document-type", "MyType") + .withOption("hosts", "http://localhost:12345") + .build()); + } + + @Test + public void validateWrongHosts() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "Could not parse host 'wrong-host' in option 'hosts'. It should follow the format 'http://host_name:port'."); + sinkFactory.createDynamicTableSink( + context() + .withOption("index", "MyIndex") + .withOption("document-type", "MyType") + .withOption("hosts", "wrong-host") + .build()); + } + + @Test + public void validateWrongFlushSize() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'sink.bulk-flush.max-size' must be in MB granularity. Got: 1024 bytes"); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), + "1kb") + .build()); + } + + @Test + public void validateWrongRetries() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("'sink.bulk-flush.backoff.max-retries' must be at least 1. Got: 0"); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION + .key(), + "0") + .build()); + } + + @Test + public void validateWrongMaxActions() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("'sink.bulk-flush.max-actions' must be at least 1. Got: -2"); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), + "-2") + .build()); + } + + @Test + public void validateWrongBackoffDelay() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("Invalid value for option 'sink.bulk-flush.backoff.delay'."); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), + "-1s") + .build()); + } + + @Test + public void validatePrimaryKeyOnIllegalColumn() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "The table has a primary key on columns of illegal types: " + + "[ARRAY, MAP, MULTISET, ROW, RAW, VARBINARY].\n" + + " Elasticsearch sink does not support primary keys on columns of types: " + + "[ARRAY, MAP, MULTISET, STRUCTURED_TYPE, ROW, RAW, BINARY, VARBINARY]."); + sinkFactory.createDynamicTableSink( + context() + .withSchema( + new ResolvedSchema( + Arrays.asList( + Column.physical("a", DataTypes.BIGINT().notNull()), + Column.physical( + "b", + DataTypes.ARRAY( + DataTypes.BIGINT() + .notNull()) + .notNull()), + Column.physical( + "c", + DataTypes.MAP( + DataTypes.BIGINT(), + DataTypes.STRING()) + .notNull()), + Column.physical( + "d", + DataTypes.MULTISET( + DataTypes.BIGINT() + .notNull()) + .notNull()), + Column.physical( + "e", + DataTypes.ROW( + DataTypes.FIELD( + "a", + DataTypes.BIGINT())) + .notNull()), + Column.physical( + "f", + DataTypes.RAW( + Void.class, + VoidSerializer.INSTANCE) + .notNull()), + Column.physical("g", DataTypes.BYTES().notNull())), + Collections.emptyList(), + UniqueConstraint.primaryKey( + "name", + Arrays.asList("a", "b", "c", "d", "e", "f", "g")))) + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), + "1s") + .build()); + } + + @Test + public void validateWrongCredential() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'username' and 'password' must be set at the same time. Got: username 'username' and password ''"); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption(ElasticsearchConnectorOptions.USERNAME_OPTION.key(), "username") + .withOption(ElasticsearchConnectorOptions.PASSWORD_OPTION.key(), "") + .build()); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java new file mode 100644 index 0000000000000..0ebc52b576483 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java @@ -0,0 +1,399 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkFunctionProvider; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.DockerImageVersions; +import org.apache.flink.util.TestLogger; + +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.client.Client; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.search.SearchHits; +import org.elasticsearch.transport.client.PreBuiltTransportClient; +import org.junit.ClassRule; +import org.junit.Test; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.utility.DockerImageName; + +import java.time.Duration; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; +import static org.apache.flink.table.api.Expressions.row; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.junit.Assert.assertThat; + +/** IT tests for {@link Elasticsearch6DynamicSink}. */ +public class Elasticsearch6DynamicSinkITCase extends TestLogger { + + @ClassRule + public static ElasticsearchContainer elasticsearchContainer = + new ElasticsearchContainer(DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_6)); + + @SuppressWarnings("deprecation") + protected final Client getClient() { + TransportAddress transportAddress = + new TransportAddress(elasticsearchContainer.getTcpHost()); + String expectedClusterName = "docker-cluster"; + Settings settings = Settings.builder().put("cluster.name", expectedClusterName).build(); + return new PreBuiltTransportClient(settings).addTransportAddress(transportAddress); + } + + @Test + public void testWritingDocuments() throws Exception { + ResolvedSchema schema = + new ResolvedSchema( + Arrays.asList( + Column.physical("a", DataTypes.BIGINT().notNull()), + Column.physical("b", DataTypes.TIME()), + Column.physical("c", DataTypes.STRING().notNull()), + Column.physical("d", DataTypes.FLOAT()), + Column.physical("e", DataTypes.TINYINT().notNull()), + Column.physical("f", DataTypes.DATE()), + Column.physical("g", DataTypes.TIMESTAMP().notNull())), + Collections.emptyList(), + UniqueConstraint.primaryKey("name", Arrays.asList("a", "g"))); + GenericRowData rowData = + GenericRowData.of( + 1L, + 12345, + StringData.fromString("ABCDE"), + 12.12f, + (byte) 2, + 12345, + TimestampData.fromLocalDateTime( + LocalDateTime.parse("2012-12-12T12:12:12"))); + + String index = "writing-documents"; + String myType = "MyType"; + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + SinkFunctionProvider sinkRuntimeProvider = + (SinkFunctionProvider) + sinkFactory + .createDynamicTableSink( + context() + .withSchema(schema) + .withOption( + ElasticsearchConnectorOptions.INDEX_OPTION + .key(), + index) + .withOption( + ElasticsearchConnectorOptions + .DOCUMENT_TYPE_OPTION + .key(), + myType) + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION + .key(), + elasticsearchContainer.getHttpHostAddress()) + .withOption( + ElasticsearchConnectorOptions + .FLUSH_ON_CHECKPOINT_OPTION + .key(), + "false") + .build()) + .getSinkRuntimeProvider(new MockContext()); + + SinkFunction sinkFunction = sinkRuntimeProvider.createSinkFunction(); + StreamExecutionEnvironment environment = + StreamExecutionEnvironment.getExecutionEnvironment(); + environment.setParallelism(4); + + rowData.setRowKind(RowKind.UPDATE_AFTER); + environment.fromElements(rowData).addSink(sinkFunction); + environment.execute(); + + Client client = getClient(); + Map response = + client.get(new GetRequest(index, myType, "1_2012-12-12T12:12:12")) + .actionGet() + .getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } + + @Test + public void testWritingDocumentsFromTableApi() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "table-api"; + String myType = "MyType"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL,\n" + + "h as a + 2,\n" + + "PRIMARY KEY (a, g) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), myType) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + + String.format( + "'%s'='%s'\n", + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), + "false") + + ")"); + + tableEnvironment + .fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("esTable") + .await(); + + Client client = getClient(); + Map response = + client.get(new GetRequest(index, myType, "1_2012-12-12T12:12:12")) + .actionGet() + .getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } + + @Test + public void testWritingDocumentsNoPrimaryKey() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "no-primary-key"; + String myType = "MyType"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), myType) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + + String.format( + "'%s'='%s'\n", + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), + "false") + + ")"); + + tableEnvironment + .fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12")), + row( + 2L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "FGHIJK", + 13.13f, + (byte) 4, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2013-12-12T13:13:13"))) + .executeInsert("esTable") + .await(); + + Client client = getClient(); + + // search API does not return documents that were not indexed, we might need to query + // the index a few times + Deadline deadline = Deadline.fromNow(Duration.ofSeconds(30)); + SearchHits hits; + do { + hits = client.prepareSearch(index).execute().actionGet().getHits(); + if (hits.getTotalHits() < 2) { + Thread.sleep(200); + } + } while (hits.getTotalHits() < 2 && deadline.hasTimeLeft()); + + if (hits.getTotalHits() < 2) { + throw new AssertionError("Could not retrieve results from Elasticsearch."); + } + + HashSet> resultSet = new HashSet<>(); + resultSet.add(hits.getAt(0).getSourceAsMap()); + resultSet.add(hits.getAt(1).getSourceAsMap()); + Map expectedMap1 = new HashMap<>(); + expectedMap1.put("a", 1); + expectedMap1.put("b", "00:00:12"); + expectedMap1.put("c", "ABCDE"); + expectedMap1.put("d", 12.12d); + expectedMap1.put("e", 2); + expectedMap1.put("f", "2003-10-20"); + expectedMap1.put("g", "2012-12-12 12:12:12"); + Map expectedMap2 = new HashMap<>(); + expectedMap2.put("a", 2); + expectedMap2.put("b", "00:00:12"); + expectedMap2.put("c", "FGHIJK"); + expectedMap2.put("d", 13.13d); + expectedMap2.put("e", 4); + expectedMap2.put("f", "2003-10-20"); + expectedMap2.put("g", "2013-12-12 13:13:13"); + HashSet> expectedSet = new HashSet<>(); + expectedSet.add(expectedMap1); + expectedSet.add(expectedMap2); + assertThat(resultSet, equalTo(expectedSet)); + } + + @Test + public void testWritingDocumentsWithDynamicIndex() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "dynamic-index-{b|yyyy-MM-dd}"; + String myType = "MyType"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIMESTAMP NOT NULL,\n" + + "PRIMARY KEY (a) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), myType) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + + String.format( + "'%s'='%s'\n", + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), + "false") + + ")"); + + tableEnvironment + .fromValues(row(1L, LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("esTable") + .await(); + + Client client = getClient(); + Map response = + client.get(new GetRequest("dynamic-index-2012-12-12", myType, "1")) + .actionGet() + .getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } + + private static class MockContext implements DynamicTableSink.Context { + @Override + public boolean isBounded() { + return false; + } + + @Override + public TypeInformation createTypeInformation(DataType consumedDataType) { + return null; + } + + @Override + public TypeInformation createTypeInformation(LogicalType consumedLogicalType) { + return null; + } + + @Override + public DynamicTableSink.DataStructureConverter createDataStructureConverter( + DataType consumedDataType) { + return null; + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java new file mode 100644 index 0000000000000..f8ab0ab0fb4b1 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java @@ -0,0 +1,298 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSink; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.util.TestLogger; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.ActionRequest; +import org.junit.Test; +import org.mockito.Mockito; + +import java.time.ZoneId; +import java.util.List; + +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; + +/** Tests for {@link Elasticsearch6DynamicSink} parameters. */ +public class Elasticsearch6DynamicSinkTest extends TestLogger { + + private static final String FIELD_KEY = "key"; + private static final String FIELD_FRUIT_NAME = "fruit_name"; + private static final String FIELD_COUNT = "count"; + private static final String FIELD_TS = "ts"; + + private static final String HOSTNAME = "host1"; + private static final int PORT = 1234; + private static final String SCHEMA = "https"; + private static final String INDEX = "MyIndex"; + private static final String DOC_TYPE = "MyType"; + private static final String USERNAME = "username"; + private static final String PASSWORD = "password"; + + @Test + public void testBuilder() { + final TableSchema schema = createTestSchema(); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch6DynamicSink testSink = + new Elasticsearch6DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch6Configuration( + getConfig(), this.getClass().getClassLoader()), + schema, + ZoneId.systemDefault(), + provider); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new DummyFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(true); + verify(provider.builderSpy) + .setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); + verify(provider.builderSpy).setBulkFlushBackoffDelay(123); + verify(provider.builderSpy).setBulkFlushBackoffRetries(3); + verify(provider.builderSpy).setBulkFlushInterval(100); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(1); + verify(provider.builderSpy) + .setRestClientFactory( + new Elasticsearch6DynamicSink.DefaultRestClientFactory("/myapp")); + verify(provider.sinkSpy).disableFlushOnCheckpoint(); + } + + @Test + public void testDefaultConfig() { + final TableSchema schema = createTestSchema(); + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + SCHEMA + "://" + HOSTNAME + ":" + PORT); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch6DynamicSink testSink = + new Elasticsearch6DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch6Configuration( + configuration, this.getClass().getClassLoader()), + schema, + ZoneId.systemDefault(), + provider); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(false); + verify(provider.builderSpy).setBulkFlushInterval(1000); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); + verify(provider.builderSpy) + .setRestClientFactory(new Elasticsearch6DynamicSink.DefaultRestClientFactory(null)); + verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); + } + + @Test + public void testAuthConfig() { + final TableSchema schema = createTestSchema(); + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + SCHEMA + "://" + HOSTNAME + ":" + PORT); + configuration.setString(ElasticsearchConnectorOptions.USERNAME_OPTION.key(), USERNAME); + configuration.setString(ElasticsearchConnectorOptions.PASSWORD_OPTION.key(), PASSWORD); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch6DynamicSink testSink = + new Elasticsearch6DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch6Configuration( + configuration, this.getClass().getClassLoader()), + schema, + ZoneId.systemDefault(), + provider); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(false); + verify(provider.builderSpy).setBulkFlushInterval(1000); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); + verify(provider.builderSpy) + .setRestClientFactory( + new Elasticsearch6DynamicSink.AuthRestClientFactory( + null, USERNAME, PASSWORD)); + verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); + } + + private Configuration getConfig() { + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + SCHEMA + "://" + HOSTNAME + ":" + PORT); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION.key(), "exponential"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "123"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), "3"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION.key(), "100"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "1000"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1mb"); + configuration.setString( + ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX.key(), "/myapp"); + configuration.setString( + ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION.key(), + DummyFailureHandler.class.getName()); + configuration.setString( + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false"); + return configuration; + } + + private static class BuilderProvider + implements Elasticsearch6DynamicSink.ElasticSearchBuilderProvider { + public ElasticsearchSink.Builder builderSpy; + public ElasticsearchSink sinkSpy; + + @Override + public ElasticsearchSink.Builder createBuilder( + List httpHosts, RowElasticsearchSinkFunction upsertSinkFunction) { + builderSpy = + Mockito.spy(new ElasticsearchSink.Builder<>(httpHosts, upsertSinkFunction)); + doAnswer( + invocation -> { + sinkSpy = + Mockito.spy( + (ElasticsearchSink) + invocation.callRealMethod()); + return sinkSpy; + }) + .when(builderSpy) + .build(); + + return builderSpy; + } + } + + private TableSchema createTestSchema() { + return TableSchema.builder() + .field(FIELD_KEY, DataTypes.BIGINT()) + .field(FIELD_FRUIT_NAME, DataTypes.STRING()) + .field(FIELD_COUNT, DataTypes.DECIMAL(10, 4)) + .field(FIELD_TS, DataTypes.TIMESTAMP(3)) + .build(); + } + + private static class DummySerializationSchema implements SerializationSchema { + + private static final DummySerializationSchema INSTANCE = new DummySerializationSchema(); + + @Override + public byte[] serialize(RowData element) { + return new byte[0]; + } + } + + private static class DummyEncodingFormat + implements EncodingFormat> { + @Override + public SerializationSchema createRuntimeEncoder( + DynamicTableSink.Context context, DataType consumedDataType) { + return DummySerializationSchema.INSTANCE; + } + + @Override + public ChangelogMode getChangelogMode() { + return null; + } + } + + private static class MockSinkContext implements DynamicTableSink.Context { + @Override + public boolean isBounded() { + return false; + } + + @Override + public TypeInformation createTypeInformation(DataType consumedDataType) { + return null; + } + + @Override + public TypeInformation createTypeInformation(LogicalType consumedLogicalType) { + return null; + } + + @Override + public DynamicTableSink.DataStructureConverter createDataStructureConverter( + DataType consumedDataType) { + return null; + } + } + + /** Custom failure handler for testing. */ + public static class DummyFailureHandler implements ActionRequestFailureHandler { + + @Override + public void onFailure( + ActionRequest action, + Throwable failure, + int restStatusCode, + RequestIndexer indexer) { + // do nothing + } + + @Override + public boolean equals(Object o) { + return o instanceof DummyFailureHandler; + } + + @Override + public int hashCode() { + return DummyFailureHandler.class.hashCode(); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index 4f01962fad790..d5d751edcbdfe 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -26,7 +26,7 @@ under the License. flink-connectors org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. @@ -196,33 +196,6 @@ under the License. 1 - - org.apache.maven.plugins - maven-shade-plugin - - - shade-flink - package - - shade - - - - - org.apache.flink:flink-connector-base - org.apache.flink:flink-connector-elasticsearch-base - - - - - org.apache.flink.connector.base - org.apache.flink.connector.elasticsearch7.shaded.org.apache.flink.connector.base - - - - - - diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java index 3a4782733842a..567f77406c19e 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java @@ -46,7 +46,6 @@ * .source(element.f1) * ); * }) - * .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE) * .build(); * } * diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java new file mode 100644 index 0000000000000..6bd28cf45539a --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.ValidationException; + +import org.apache.http.HttpHost; + +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; + +/** Elasticsearch 7 specific configuration. */ +@Internal +final class Elasticsearch7Configuration extends ElasticsearchConfiguration { + Elasticsearch7Configuration(ReadableConfig config, ClassLoader classLoader) { + super(config, classLoader); + } + + public List getHosts() { + return config.get(HOSTS_OPTION).stream() + .map(Elasticsearch7Configuration::validateAndParseHostsString) + .collect(Collectors.toList()); + } + + private static HttpHost validateAndParseHostsString(String host) { + try { + HttpHost httpHost = HttpHost.create(host); + if (httpHost.getPort() < 0) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing port.", + host, HOSTS_OPTION.key())); + } + + if (httpHost.getSchemeName() == null) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing scheme.", + host, HOSTS_OPTION.key())); + } + return httpHost; + } catch (Exception e) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'.", + host, HOSTS_OPTION.key()), + e); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java new file mode 100644 index 0000000000000..1926e445986be --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java @@ -0,0 +1,335 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.connectors.elasticsearch7.ElasticsearchSink; +import org.apache.flink.streaming.connectors.elasticsearch7.RestClientFactory; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkFunctionProvider; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.StringUtils; + +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.common.xcontent.XContentType; + +import javax.annotation.Nullable; + +import java.time.ZoneId; +import java.util.List; +import java.util.Objects; + +/** + * A {@link DynamicTableSink} that describes how to create a {@link ElasticsearchSink} from a + * logical description. + */ +@Internal +final class Elasticsearch7DynamicSink implements DynamicTableSink { + @VisibleForTesting + static final Elasticsearch7RequestFactory REQUEST_FACTORY = new Elasticsearch7RequestFactory(); + + private final EncodingFormat> format; + private final TableSchema schema; + private final Elasticsearch7Configuration config; + private final ZoneId localTimeZoneId; + private final boolean isDynamicIndexWithSystemTime; + + public Elasticsearch7DynamicSink( + EncodingFormat> format, + Elasticsearch7Configuration config, + TableSchema schema, + ZoneId localTimeZoneId) { + this(format, config, schema, localTimeZoneId, (ElasticsearchSink.Builder::new)); + } + + // -------------------------------------------------------------- + // Hack to make configuration testing possible. + // + // The code in this block should never be used outside of tests. + // Having a way to inject a builder we can assert the builder in + // the test. We can not assert everything though, e.g. it is not + // possible to assert flushing on checkpoint, as it is configured + // on the sink itself. + // -------------------------------------------------------------- + + private final ElasticSearchBuilderProvider builderProvider; + + @FunctionalInterface + interface ElasticSearchBuilderProvider { + ElasticsearchSink.Builder createBuilder( + List httpHosts, RowElasticsearchSinkFunction upsertSinkFunction); + } + + Elasticsearch7DynamicSink( + EncodingFormat> format, + Elasticsearch7Configuration config, + TableSchema schema, + ZoneId localTimeZoneId, + ElasticSearchBuilderProvider builderProvider) { + this.format = format; + this.schema = schema; + this.config = config; + this.localTimeZoneId = localTimeZoneId; + this.isDynamicIndexWithSystemTime = isDynamicIndexWithSystemTime(); + this.builderProvider = builderProvider; + } + + // -------------------------------------------------------------- + // End of hack to make configuration testing possible + // -------------------------------------------------------------- + + public boolean isDynamicIndexWithSystemTime() { + IndexGeneratorFactory.IndexHelper indexHelper = new IndexGeneratorFactory.IndexHelper(); + return indexHelper.checkIsDynamicIndexWithSystemTimeFormat(config.getIndex()); + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { + ChangelogMode.Builder builder = ChangelogMode.newBuilder(); + for (RowKind kind : requestedMode.getContainedKinds()) { + if (kind != RowKind.UPDATE_BEFORE) { + builder.addContainedKind(kind); + } + } + if (isDynamicIndexWithSystemTime && !requestedMode.containsOnly(RowKind.INSERT)) { + throw new ValidationException( + "Dynamic indexing based on system time only works on append only stream."); + } + return builder.build(); + } + + @Override + public SinkFunctionProvider getSinkRuntimeProvider(Context context) { + return () -> { + SerializationSchema format = + this.format.createRuntimeEncoder(context, schema.toRowDataType()); + + final RowElasticsearchSinkFunction upsertFunction = + new RowElasticsearchSinkFunction( + IndexGeneratorFactory.createIndexGenerator( + config.getIndex(), schema, localTimeZoneId), + null, // this is deprecated in es 7+ + format, + XContentType.JSON, + REQUEST_FACTORY, + KeyExtractor.createKeyExtractor(schema, config.getKeyDelimiter())); + + final ElasticsearchSink.Builder builder = + builderProvider.createBuilder(config.getHosts(), upsertFunction); + + builder.setFailureHandler(config.getFailureHandler()); + builder.setBulkFlushMaxActions(config.getBulkFlushMaxActions()); + builder.setBulkFlushMaxSizeMb((int) (config.getBulkFlushMaxByteSize() >> 20)); + builder.setBulkFlushInterval(config.getBulkFlushInterval()); + builder.setBulkFlushBackoff(config.isBulkFlushBackoffEnabled()); + config.getBulkFlushBackoffType().ifPresent(builder::setBulkFlushBackoffType); + config.getBulkFlushBackoffRetries().ifPresent(builder::setBulkFlushBackoffRetries); + config.getBulkFlushBackoffDelay().ifPresent(builder::setBulkFlushBackoffDelay); + + // we must overwrite the default factory which is defined with a lambda because of a bug + // in shading lambda serialization shading see FLINK-18006 + if (config.getUsername().isPresent() + && config.getPassword().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get()) + && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get())) { + builder.setRestClientFactory( + new AuthRestClientFactory( + config.getPathPrefix().orElse(null), + config.getUsername().get(), + config.getPassword().get())); + } else { + builder.setRestClientFactory( + new DefaultRestClientFactory(config.getPathPrefix().orElse(null))); + } + + final ElasticsearchSink sink = builder.build(); + + if (config.isDisableFlushOnCheckpoint()) { + sink.disableFlushOnCheckpoint(); + } + + return sink; + }; + } + + @Override + public DynamicTableSink copy() { + return this; + } + + @Override + public String asSummaryString() { + return "Elasticsearch7"; + } + + /** Serializable {@link RestClientFactory} used by the sink. */ + @VisibleForTesting + static class DefaultRestClientFactory implements RestClientFactory { + + private final String pathPrefix; + + public DefaultRestClientFactory(@Nullable String pathPrefix) { + this.pathPrefix = pathPrefix; + } + + @Override + public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { + if (pathPrefix != null) { + restClientBuilder.setPathPrefix(pathPrefix); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DefaultRestClientFactory that = (DefaultRestClientFactory) o; + return Objects.equals(pathPrefix, that.pathPrefix); + } + + @Override + public int hashCode() { + return Objects.hash(pathPrefix); + } + } + + /** Serializable {@link RestClientFactory} used by the sink which enable authentication. */ + @VisibleForTesting + static class AuthRestClientFactory implements RestClientFactory { + + private final String pathPrefix; + private final String username; + private final String password; + private transient CredentialsProvider credentialsProvider; + + public AuthRestClientFactory( + @Nullable String pathPrefix, String username, String password) { + this.pathPrefix = pathPrefix; + this.password = password; + this.username = username; + } + + @Override + public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { + if (pathPrefix != null) { + restClientBuilder.setPathPrefix(pathPrefix); + } + if (credentialsProvider == null) { + credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials( + AuthScope.ANY, new UsernamePasswordCredentials(username, password)); + } + restClientBuilder.setHttpClientConfigCallback( + httpAsyncClientBuilder -> + httpAsyncClientBuilder.setDefaultCredentialsProvider( + credentialsProvider)); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + AuthRestClientFactory that = (AuthRestClientFactory) o; + return Objects.equals(pathPrefix, that.pathPrefix) + && Objects.equals(username, that.username) + && Objects.equals(password, that.password); + } + + @Override + public int hashCode() { + return Objects.hash(pathPrefix, password, username); + } + } + + /** + * Version-specific creation of {@link org.elasticsearch.action.ActionRequest}s used by the + * sink. + */ + private static class Elasticsearch7RequestFactory implements RequestFactory { + @Override + public UpdateRequest createUpdateRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document) { + return new UpdateRequest(index, key) + .doc(document, contentType) + .upsert(document, contentType); + } + + @Override + public IndexRequest createIndexRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document) { + return new IndexRequest(index).id(key).source(document, contentType); + } + + @Override + public DeleteRequest createDeleteRequest(String index, String docType, String key) { + return new DeleteRequest(index, key); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Elasticsearch7DynamicSink that = (Elasticsearch7DynamicSink) o; + return Objects.equals(format, that.format) + && Objects.equals(schema, that.schema) + && Objects.equals(config, that.config) + && Objects.equals(builderProvider, that.builderProvider); + } + + @Override + public int hashCode() { + return Objects.hash(format, schema, config, builderProvider); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java new file mode 100644 index 0000000000000..70b0134d226d5 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.utils.TableSchemaUtils; +import org.apache.flink.util.StringUtils; + +import java.time.ZoneId; +import java.util.Set; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FORMAT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.INDEX_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; + +/** A {@link DynamicTableSinkFactory} for discovering {@link Elasticsearch7DynamicSink}. */ +@Internal +public class Elasticsearch7DynamicSinkFactory implements DynamicTableSinkFactory { + private static final Set> requiredOptions = + Stream.of(HOSTS_OPTION, INDEX_OPTION).collect(Collectors.toSet()); + private static final Set> optionalOptions = + Stream.of( + KEY_DELIMITER_OPTION, + FAILURE_HANDLER_OPTION, + FLUSH_ON_CHECKPOINT_OPTION, + BULK_FLASH_MAX_SIZE_OPTION, + BULK_FLUSH_MAX_ACTIONS_OPTION, + BULK_FLUSH_INTERVAL_OPTION, + BULK_FLUSH_BACKOFF_TYPE_OPTION, + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION, + BULK_FLUSH_BACKOFF_DELAY_OPTION, + CONNECTION_PATH_PREFIX, + FORMAT_OPTION, + PASSWORD_OPTION, + USERNAME_OPTION) + .collect(Collectors.toSet()); + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + TableSchema tableSchema = context.getCatalogTable().getSchema(); + ElasticsearchValidationUtils.validatePrimaryKey(tableSchema); + + final FactoryUtil.TableFactoryHelper helper = + FactoryUtil.createTableFactoryHelper(this, context); + + final EncodingFormat> format = + helper.discoverEncodingFormat(SerializationFormatFactory.class, FORMAT_OPTION); + + helper.validate(); + Configuration configuration = new Configuration(); + context.getCatalogTable().getOptions().forEach(configuration::setString); + Elasticsearch7Configuration config = + new Elasticsearch7Configuration(configuration, context.getClassLoader()); + + validate(config, configuration); + + return new Elasticsearch7DynamicSink( + format, + config, + TableSchemaUtils.getPhysicalSchema(tableSchema), + getLocalTimeZoneId(context.getConfiguration())); + } + + ZoneId getLocalTimeZoneId(ReadableConfig readableConfig) { + final String zone = readableConfig.get(TableConfigOptions.LOCAL_TIME_ZONE); + final ZoneId zoneId = + TableConfigOptions.LOCAL_TIME_ZONE.defaultValue().equals(zone) + ? ZoneId.systemDefault() + : ZoneId.of(zone); + + return zoneId; + } + + private void validate(Elasticsearch7Configuration config, Configuration originalConfiguration) { + config.getFailureHandler(); // checks if we can instantiate the custom failure handler + config.getHosts(); // validate hosts + validate( + config.getIndex().length() >= 1, + () -> String.format("'%s' must not be empty", INDEX_OPTION.key())); + int maxActions = config.getBulkFlushMaxActions(); + validate( + maxActions == -1 || maxActions >= 1, + () -> + String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_MAX_ACTIONS_OPTION.key(), maxActions)); + long maxSize = config.getBulkFlushMaxByteSize(); + long mb1 = 1024 * 1024; + validate( + maxSize == -1 || (maxSize >= mb1 && maxSize % mb1 == 0), + () -> + String.format( + "'%s' must be in MB granularity. Got: %s", + BULK_FLASH_MAX_SIZE_OPTION.key(), + originalConfiguration + .get(BULK_FLASH_MAX_SIZE_OPTION) + .toHumanReadableString())); + validate( + config.getBulkFlushBackoffRetries().map(retries -> retries >= 1).orElse(true), + () -> + String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), + config.getBulkFlushBackoffRetries().get())); + if (config.getUsername().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) { + validate( + config.getPassword().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get()), + () -> + String.format( + "'%s' and '%s' must be set at the same time. Got: username '%s' and password '%s'", + USERNAME_OPTION.key(), + PASSWORD_OPTION.key(), + config.getUsername().get(), + config.getPassword().orElse(""))); + } + } + + private static void validate(boolean condition, Supplier message) { + if (!condition) { + throw new ValidationException(message.get()); + } + } + + @Override + public String factoryIdentifier() { + return "elasticsearch-7"; + } + + @Override + public Set> requiredOptions() { + return requiredOptions; + } + + @Override + public Set> optionalOptions() { + return optionalOptions; + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory index 9e189e38caf4e..10e4846e5ed06 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.connector.elasticsearch.table.Elasticsearch7DynamicSinkFactory +org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7DynamicSinkFactory diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java new file mode 100644 index 0000000000000..3ff21ed1ab4bd --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java @@ -0,0 +1,234 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.api.common.typeutils.base.VoidSerializer; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.util.TestLogger; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.Arrays; +import java.util.Collections; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; + +/** Tests for validation in {@link Elasticsearch7DynamicSinkFactory}. */ +public class Elasticsearch7DynamicSinkFactoryTest extends TestLogger { + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Test + public void validateEmptyConfiguration() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "One or more required options are missing.\n" + + "\n" + + "Missing required options are:\n" + + "\n" + + "hosts\n" + + "index"); + sinkFactory.createDynamicTableSink(context().build()); + } + + @Test + public void validateWrongIndex() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("'index' must not be empty"); + sinkFactory.createDynamicTableSink( + context() + .withOption("index", "") + .withOption("hosts", "http://localhost:12345") + .build()); + } + + @Test + public void validateWrongHosts() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "Could not parse host 'wrong-host' in option 'hosts'. It should follow the format 'http://host_name:port'."); + sinkFactory.createDynamicTableSink( + context().withOption("index", "MyIndex").withOption("hosts", "wrong-host").build()); + } + + @Test + public void validateWrongFlushSize() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'sink.bulk-flush.max-size' must be in MB granularity. Got: 1024 bytes"); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), + "1kb") + .build()); + } + + @Test + public void validateWrongRetries() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("'sink.bulk-flush.backoff.max-retries' must be at least 1. Got: 0"); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION + .key(), + "0") + .build()); + } + + @Test + public void validateWrongMaxActions() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("'sink.bulk-flush.max-actions' must be at least 1. Got: -2"); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), + "-2") + .build()); + } + + @Test + public void validateWrongBackoffDelay() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("Invalid value for option 'sink.bulk-flush.backoff.delay'."); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), + "-1s") + .build()); + } + + @Test + public void validatePrimaryKeyOnIllegalColumn() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "The table has a primary key on columns of illegal types: " + + "[ARRAY, MAP, MULTISET, ROW, RAW, VARBINARY].\n" + + " Elasticsearch sink does not support primary keys on columns of types: " + + "[ARRAY, MAP, MULTISET, STRUCTURED_TYPE, ROW, RAW, BINARY, VARBINARY]."); + sinkFactory.createDynamicTableSink( + context() + .withSchema( + new ResolvedSchema( + Arrays.asList( + Column.physical("a", DataTypes.BIGINT().notNull()), + Column.physical( + "b", + DataTypes.ARRAY( + DataTypes.BIGINT() + .notNull()) + .notNull()), + Column.physical( + "c", + DataTypes.MAP( + DataTypes.BIGINT(), + DataTypes.STRING()) + .notNull()), + Column.physical( + "d", + DataTypes.MULTISET( + DataTypes.BIGINT() + .notNull()) + .notNull()), + Column.physical( + "e", + DataTypes.ROW( + DataTypes.FIELD( + "a", + DataTypes.BIGINT())) + .notNull()), + Column.physical( + "f", + DataTypes.RAW( + Void.class, + VoidSerializer.INSTANCE) + .notNull()), + Column.physical("g", DataTypes.BYTES().notNull())), + Collections.emptyList(), + UniqueConstraint.primaryKey( + "name", + Arrays.asList("a", "b", "c", "d", "e", "f", "g")))) + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), + "1s") + .build()); + } + + @Test + public void validateWrongCredential() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'username' and 'password' must be set at the same time. Got: username 'username' and password ''"); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption(ElasticsearchConnectorOptions.USERNAME_OPTION.key(), "username") + .withOption(ElasticsearchConnectorOptions.PASSWORD_OPTION.key(), "") + .build()); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java new file mode 100644 index 0000000000000..fdf7a3a09f3bc --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -0,0 +1,376 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkFunctionProvider; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.DockerImageVersions; +import org.apache.flink.util.TestLogger; + +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.client.Client; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.search.SearchHits; +import org.elasticsearch.transport.client.PreBuiltTransportClient; +import org.junit.ClassRule; +import org.junit.Test; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.utility.DockerImageName; + +import java.time.Duration; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; +import static org.apache.flink.table.api.Expressions.row; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.junit.Assert.assertThat; + +/** IT tests for {@link Elasticsearch7DynamicSink}. */ +public class Elasticsearch7DynamicSinkITCase extends TestLogger { + + @ClassRule + public static ElasticsearchContainer elasticsearchContainer = + new ElasticsearchContainer(DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_7)); + + @SuppressWarnings("deprecation") + protected final Client getClient() { + TransportAddress transportAddress = + new TransportAddress(elasticsearchContainer.getTcpHost()); + String expectedClusterName = "docker-cluster"; + Settings settings = Settings.builder().put("cluster.name", expectedClusterName).build(); + return new PreBuiltTransportClient(settings).addTransportAddress(transportAddress); + } + + @Test + public void testWritingDocuments() throws Exception { + ResolvedSchema schema = + new ResolvedSchema( + Arrays.asList( + Column.physical("a", DataTypes.BIGINT().notNull()), + Column.physical("b", DataTypes.TIME()), + Column.physical("c", DataTypes.STRING().notNull()), + Column.physical("d", DataTypes.FLOAT()), + Column.physical("e", DataTypes.TINYINT().notNull()), + Column.physical("f", DataTypes.DATE()), + Column.physical("g", DataTypes.TIMESTAMP().notNull())), + Collections.emptyList(), + UniqueConstraint.primaryKey("name", Arrays.asList("a", "g"))); + + GenericRowData rowData = + GenericRowData.of( + 1L, + 12345, + StringData.fromString("ABCDE"), + 12.12f, + (byte) 2, + 12345, + TimestampData.fromLocalDateTime( + LocalDateTime.parse("2012-12-12T12:12:12"))); + + String index = "writing-documents"; + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + SinkFunctionProvider sinkRuntimeProvider = + (SinkFunctionProvider) + sinkFactory + .createDynamicTableSink( + context() + .withSchema(schema) + .withOption( + ElasticsearchConnectorOptions.INDEX_OPTION + .key(), + index) + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION + .key(), + elasticsearchContainer.getHttpHostAddress()) + .withOption( + ElasticsearchConnectorOptions + .FLUSH_ON_CHECKPOINT_OPTION + .key(), + "false") + .build()) + .getSinkRuntimeProvider(new MockContext()); + + SinkFunction sinkFunction = sinkRuntimeProvider.createSinkFunction(); + StreamExecutionEnvironment environment = + StreamExecutionEnvironment.getExecutionEnvironment(); + environment.setParallelism(4); + + rowData.setRowKind(RowKind.UPDATE_AFTER); + environment.fromElements(rowData).addSink(sinkFunction); + environment.execute(); + + Client client = getClient(); + Map response = + client.get(new GetRequest(index, "1_2012-12-12T12:12:12")).actionGet().getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } + + @Test + public void testWritingDocumentsFromTableApi() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "table-api"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL," + + "h as a + 2,\n" + + "PRIMARY KEY (a, g) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + + String.format( + "'%s'='%s'\n", + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), + "false") + + ")"); + + tableEnvironment + .fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("esTable") + .await(); + + Client client = getClient(); + Map response = + client.get(new GetRequest(index, "1_2012-12-12T12:12:12")).actionGet().getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } + + @Test + public void testWritingDocumentsNoPrimaryKey() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "no-primary-key"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + + String.format( + "'%s'='%s'\n", + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), + "false") + + ")"); + + tableEnvironment + .fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12")), + row( + 2L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "FGHIJK", + 13.13f, + (byte) 4, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2013-12-12T13:13:13"))) + .executeInsert("esTable") + .await(); + + Client client = getClient(); + + // search API does not return documents that were not indexed, we might need to query + // the index a few times + Deadline deadline = Deadline.fromNow(Duration.ofSeconds(30)); + SearchHits hits; + do { + hits = client.prepareSearch(index).execute().actionGet().getHits(); + if (hits.getTotalHits().value < 2) { + Thread.sleep(200); + } + } while (hits.getTotalHits().value < 2 && deadline.hasTimeLeft()); + + if (hits.getTotalHits().value < 2) { + throw new AssertionError("Could not retrieve results from Elasticsearch."); + } + + HashSet> resultSet = new HashSet<>(); + resultSet.add(hits.getAt(0).getSourceAsMap()); + resultSet.add(hits.getAt(1).getSourceAsMap()); + Map expectedMap1 = new HashMap<>(); + expectedMap1.put("a", 1); + expectedMap1.put("b", "00:00:12"); + expectedMap1.put("c", "ABCDE"); + expectedMap1.put("d", 12.12d); + expectedMap1.put("e", 2); + expectedMap1.put("f", "2003-10-20"); + expectedMap1.put("g", "2012-12-12 12:12:12"); + Map expectedMap2 = new HashMap<>(); + expectedMap2.put("a", 2); + expectedMap2.put("b", "00:00:12"); + expectedMap2.put("c", "FGHIJK"); + expectedMap2.put("d", 13.13d); + expectedMap2.put("e", 4); + expectedMap2.put("f", "2003-10-20"); + expectedMap2.put("g", "2013-12-12 13:13:13"); + HashSet> expectedSet = new HashSet<>(); + expectedSet.add(expectedMap1); + expectedSet.add(expectedMap2); + assertThat(resultSet, equalTo(expectedSet)); + } + + @Test + public void testWritingDocumentsWithDynamicIndex() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "dynamic-index-{b|yyyy-MM-dd}"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIMESTAMP NOT NULL,\n" + + "PRIMARY KEY (a) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + + String.format( + "'%s'='%s'\n", + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), + "false") + + ")"); + + tableEnvironment + .fromValues(row(1L, LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("esTable") + .await(); + + Client client = getClient(); + Map response = + client.get(new GetRequest("dynamic-index-2012-12-12", "1")).actionGet().getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } + + private static class MockContext implements DynamicTableSink.Context { + @Override + public boolean isBounded() { + return false; + } + + @Override + public TypeInformation createTypeInformation(DataType consumedDataType) { + return null; + } + + @Override + public TypeInformation createTypeInformation(LogicalType consumedLogicalType) { + return null; + } + + @Override + public DynamicTableSink.DataStructureConverter createDataStructureConverter( + DataType consumedDataType) { + return null; + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java new file mode 100644 index 0000000000000..2928c8dba6ee5 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java @@ -0,0 +1,298 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch7.ElasticsearchSink; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.util.TestLogger; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.ActionRequest; +import org.junit.Test; +import org.mockito.Mockito; + +import java.time.ZoneId; +import java.util.List; + +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; + +/** Tests for {@link Elasticsearch7DynamicSink} parameters. */ +public class Elasticsearch7DynamicSinkTest extends TestLogger { + + private static final String FIELD_KEY = "key"; + private static final String FIELD_FRUIT_NAME = "fruit_name"; + private static final String FIELD_COUNT = "count"; + private static final String FIELD_TS = "ts"; + + private static final String HOSTNAME = "host1"; + private static final int PORT = 1234; + private static final String SCHEMA = "https"; + private static final String INDEX = "MyIndex"; + private static final String DOC_TYPE = "MyType"; + private static final String USERNAME = "username"; + private static final String PASSWORD = "password"; + + @Test + public void testBuilder() { + final TableSchema schema = createTestSchema(); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch7DynamicSink testSink = + new Elasticsearch7DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch7Configuration( + getConfig(), this.getClass().getClassLoader()), + schema, + ZoneId.systemDefault(), + provider); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new DummyFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(true); + verify(provider.builderSpy) + .setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); + verify(provider.builderSpy).setBulkFlushBackoffDelay(123); + verify(provider.builderSpy).setBulkFlushBackoffRetries(3); + verify(provider.builderSpy).setBulkFlushInterval(100); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(1); + verify(provider.builderSpy) + .setRestClientFactory( + new Elasticsearch7DynamicSink.DefaultRestClientFactory("/myapp")); + verify(provider.sinkSpy).disableFlushOnCheckpoint(); + } + + @Test + public void testDefaultConfig() { + final TableSchema schema = createTestSchema(); + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + SCHEMA + "://" + HOSTNAME + ":" + PORT); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch7DynamicSink testSink = + new Elasticsearch7DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch7Configuration( + configuration, this.getClass().getClassLoader()), + schema, + ZoneId.systemDefault(), + provider); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(false); + verify(provider.builderSpy).setBulkFlushInterval(1000); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); + verify(provider.builderSpy) + .setRestClientFactory(new Elasticsearch7DynamicSink.DefaultRestClientFactory(null)); + verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); + } + + @Test + public void testAuthConfig() { + final TableSchema schema = createTestSchema(); + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + SCHEMA + "://" + HOSTNAME + ":" + PORT); + configuration.setString(ElasticsearchConnectorOptions.USERNAME_OPTION.key(), USERNAME); + configuration.setString(ElasticsearchConnectorOptions.PASSWORD_OPTION.key(), PASSWORD); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch7DynamicSink testSink = + new Elasticsearch7DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch7Configuration( + configuration, this.getClass().getClassLoader()), + schema, + ZoneId.systemDefault(), + provider); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(false); + verify(provider.builderSpy).setBulkFlushInterval(1000); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); + verify(provider.builderSpy) + .setRestClientFactory( + new Elasticsearch7DynamicSink.AuthRestClientFactory( + null, USERNAME, PASSWORD)); + verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); + } + + private Configuration getConfig() { + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + SCHEMA + "://" + HOSTNAME + ":" + PORT); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION.key(), "exponential"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "123"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), "3"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION.key(), "100"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "1000"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1mb"); + configuration.setString( + ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX.key(), "/myapp"); + configuration.setString( + ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION.key(), + DummyFailureHandler.class.getName()); + configuration.setString( + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false"); + return configuration; + } + + private static class BuilderProvider + implements Elasticsearch7DynamicSink.ElasticSearchBuilderProvider { + public ElasticsearchSink.Builder builderSpy; + public ElasticsearchSink sinkSpy; + + @Override + public ElasticsearchSink.Builder createBuilder( + List httpHosts, RowElasticsearchSinkFunction upsertSinkFunction) { + builderSpy = + Mockito.spy(new ElasticsearchSink.Builder<>(httpHosts, upsertSinkFunction)); + doAnswer( + invocation -> { + sinkSpy = + Mockito.spy( + (ElasticsearchSink) + invocation.callRealMethod()); + return sinkSpy; + }) + .when(builderSpy) + .build(); + + return builderSpy; + } + } + + private TableSchema createTestSchema() { + return TableSchema.builder() + .field(FIELD_KEY, DataTypes.BIGINT()) + .field(FIELD_FRUIT_NAME, DataTypes.STRING()) + .field(FIELD_COUNT, DataTypes.DECIMAL(10, 4)) + .field(FIELD_TS, DataTypes.TIMESTAMP(3)) + .build(); + } + + private static class DummySerializationSchema implements SerializationSchema { + + private static final DummySerializationSchema INSTANCE = new DummySerializationSchema(); + + @Override + public byte[] serialize(RowData element) { + return new byte[0]; + } + } + + private static class DummyEncodingFormat + implements EncodingFormat> { + @Override + public SerializationSchema createRuntimeEncoder( + DynamicTableSink.Context context, DataType consumedDataType) { + return DummySerializationSchema.INSTANCE; + } + + @Override + public ChangelogMode getChangelogMode() { + return null; + } + } + + private static class MockSinkContext implements DynamicTableSink.Context { + @Override + public boolean isBounded() { + return false; + } + + @Override + public TypeInformation createTypeInformation(DataType consumedDataType) { + return null; + } + + @Override + public TypeInformation createTypeInformation(LogicalType consumedLogicalType) { + return null; + } + + @Override + public DynamicTableSink.DataStructureConverter createDataStructureConverter( + DataType consumedDataType) { + return null; + } + } + + /** Custom failure handler for testing. */ + public static class DummyFailureHandler implements ActionRequestFailureHandler { + + @Override + public void onFailure( + ActionRequest action, + Throwable failure, + int restStatusCode, + RequestIndexer indexer) { + // do nothing + } + + @Override + public boolean equals(Object o) { + return o instanceof DummyFailureHandler; + } + + @Override + public int hashCode() { + return DummyFailureHandler.class.hashCode(); + } + } +} diff --git a/flink-connectors/flink-connector-files/pom.xml b/flink-connectors/flink-connector-files/pom.xml index e49cc84e2ecda..ec887187430d4 100644 --- a/flink-connectors/flink-connector-files/pom.xml +++ b/flink-connectors/flink-connector-files/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. @@ -152,12 +152,6 @@ under the License. org.apache.flink:flink-connector-base - - - org.apache.flink.connector.base - org.apache.flink.connector.files.shaded.org.apache.flink.connector.base - - diff --git a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/sink/writer/FileWriter.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/sink/writer/FileWriter.java index fad3b50bb6603..51cc6d8c10f64 100644 --- a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/sink/writer/FileWriter.java +++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/sink/writer/FileWriter.java @@ -92,7 +92,7 @@ public class FileWriter private final OutputFileConfig outputFileConfig; - private final Counter recordsOutCounter; + private final Counter numRecordsSendCounter; private boolean endOfInput; @@ -128,8 +128,7 @@ public FileWriter( this.activeBuckets = new HashMap<>(); this.bucketerContext = new BucketerContext(); - this.recordsOutCounter = - checkNotNull(metricGroup).getIOMetricGroup().getNumRecordsOutCounter(); + this.numRecordsSendCounter = checkNotNull(metricGroup).getNumRecordsSendCounter(); this.processingTimeService = checkNotNull(processingTimeService); checkArgument( bucketCheckInterval > 0, @@ -196,7 +195,7 @@ public void write(IN element, Context context) throws IOException, InterruptedEx final String bucketId = bucketAssigner.getBucketId(element, bucketerContext); final FileWriterBucket bucket = getOrCreateBucketForBucketId(bucketId); bucket.write(element, processingTimeService.getCurrentProcessingTime()); - recordsOutCounter.inc(); + numRecordsSendCounter.inc(); } @Override diff --git a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableSource.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableSource.java index 5fc8dd1e71827..448a39618ed0d 100644 --- a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableSource.java +++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableSource.java @@ -64,8 +64,11 @@ import java.util.Objects; import java.util.Optional; import java.util.stream.Collectors; +import java.util.stream.IntStream; import java.util.stream.Stream; +import static org.apache.flink.util.CollectionUtil.entry; + /** File system table source. */ @Internal public class FileSystemTableSource extends AbstractFileSystemTable @@ -109,20 +112,14 @@ public FileSystemTableSource( @Override public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) { - // When this table has no partition, just return a empty source. + // When this table has no partition, just return an empty source. if (!partitionKeys.isEmpty() && getOrFetchPartitions().isEmpty()) { return InputFormatProvider.of(new CollectionInputFormat<>(new ArrayList<>(), null)); } // Resolve metadata and make sure to filter out metadata not in the producedDataType final List metadataKeys = - DataType.getFieldNames(producedDataType).stream() - .filter( - ((this.metadataKeys == null) - ? Collections.emptyList() - : this.metadataKeys) - ::contains) - .collect(Collectors.toList()); + this.metadataKeys == null ? Collections.emptyList() : this.metadataKeys; final List metadataToExtract = metadataKeys.stream().map(ReadableFileInfo::resolve).collect(Collectors.toList()); @@ -225,16 +222,27 @@ private BulkFormat wrapBulkFormat( List metadata, List partitionKeys) { if (!metadata.isEmpty() || !partitionKeys.isEmpty()) { + final List producedFieldNames = DataType.getFieldNames(producedDataType); + final Map metadataColumns = + IntStream.range(0, metadata.size()) + .mapToObj( + i -> { + // Access metadata columns from the back because the + // names are decided by the planner + final int columnPos = + producedFieldNames.size() - metadata.size() + i; + return entry( + producedFieldNames.get(columnPos), + metadata.get(i).getAccessor()); + }) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + bulkFormat = new FileInfoExtractorBulkFormat( bulkFormat, producedDataType, context.createTypeInformation(producedDataType), - metadata.stream() - .collect( - Collectors.toMap( - ReadableFileInfo::getKey, - ReadableFileInfo::getAccessor)), + metadataColumns, partitionKeys, defaultPartName); } diff --git a/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/writer/FileWriterTest.java b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/writer/FileWriterTest.java index e521f611cfa6b..c0a5ac6677688 100644 --- a/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/writer/FileWriterTest.java +++ b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/writer/FileWriterTest.java @@ -42,13 +42,10 @@ import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.OnCheckpointRollingPolicy; import org.apache.flink.util.ExceptionUtils; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; -import java.io.File; import java.io.IOException; import java.time.Duration; import java.util.ArrayList; @@ -62,27 +59,21 @@ import java.util.Queue; import java.util.concurrent.ScheduledFuture; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link FileWriter}. */ public class FileWriterTest { - @ClassRule public static final TemporaryFolder TEMP_FOLDER = new TemporaryFolder(); - private MetricListener metricListener; - @Before + @BeforeEach public void setUp() { metricListener = new MetricListener(); } @Test - public void testPreCommit() throws Exception { - File outDir = TEMP_FOLDER.newFolder(); - Path path = new Path(outDir.toURI()); + public void testPreCommit(@TempDir java.nio.file.Path tempDir) throws Exception { + Path path = new Path(tempDir.toUri()); FileWriter fileWriter = createWriter( @@ -95,13 +86,13 @@ public void testPreCommit() throws Exception { fileWriter.write("test3", new ContextImpl()); Collection committables = fileWriter.prepareCommit(); - assertEquals(3, committables.size()); + + assertThat(committables.size()).isEqualTo(3); } @Test - public void testSnapshotAndRestore() throws Exception { - File outDir = TEMP_FOLDER.newFolder(); - Path path = new Path(outDir.toURI()); + public void testSnapshotAndRestore(@TempDir java.nio.file.Path tempDir) throws Exception { + Path path = new Path(tempDir.toUri()); FileWriter fileWriter = createWriter( @@ -112,11 +103,12 @@ public void testSnapshotAndRestore() throws Exception { fileWriter.write("test1", new ContextImpl()); fileWriter.write("test2", new ContextImpl()); fileWriter.write("test3", new ContextImpl()); - assertEquals(3, fileWriter.getActiveBuckets().size()); + assertThat(fileWriter.getActiveBuckets().size()).isEqualTo(3); fileWriter.prepareCommit(); List states = fileWriter.snapshotState(1L); - assertEquals(3, states.size()); + + assertThat(states.size()).isEqualTo(3); fileWriter = restoreWriter( @@ -124,18 +116,20 @@ public void testSnapshotAndRestore() throws Exception { path, OnCheckpointRollingPolicy.build(), new OutputFileConfig("part-", "")); - assertEquals( - fileWriter.getActiveBuckets().keySet(), - new HashSet<>(Arrays.asList("test1", "test2", "test3"))); + + assertThat(fileWriter.getActiveBuckets().keySet()) + .isEqualTo(new HashSet<>(Arrays.asList("test1", "test2", "test3"))); + for (FileWriterBucket bucket : fileWriter.getActiveBuckets().values()) { - assertNotNull("The in-progress file should be recovered", bucket.getInProgressPart()); + assertThat(bucket.getInProgressPart()) + .as("The in-progress file should be recovered") + .isNotNull(); } } @Test - public void testMergingForRescaling() throws Exception { - File outDir = TEMP_FOLDER.newFolder(); - Path path = new Path(outDir.toURI()); + public void testMergingForRescaling(@TempDir java.nio.file.Path tempDir) throws Exception { + Path path = new Path(tempDir.toUri()); FileWriter firstFileWriter = createWriter( @@ -172,27 +166,34 @@ public void testMergingForRescaling() throws Exception { path, DefaultRollingPolicy.builder().build(), new OutputFileConfig("part-", "")); - assertEquals(3, restoredWriter.getActiveBuckets().size()); + + assertThat(restoredWriter.getActiveBuckets().size()).isEqualTo(3); // Merged buckets for (String bucketId : Arrays.asList("test1", "test2")) { FileWriterBucket bucket = restoredWriter.getActiveBuckets().get(bucketId); - assertNotNull("The in-progress file should be recovered", bucket.getInProgressPart()); - assertEquals(1, bucket.getPendingFiles().size()); + + assertThat(bucket.getInProgressPart()) + .as("The in-progress file should be recovered") + .isNotNull(); + assertThat(bucket.getPendingFiles().size()).isEqualTo(1); } // Not merged buckets for (String bucketId : Collections.singletonList("test3")) { FileWriterBucket bucket = restoredWriter.getActiveBuckets().get(bucketId); - assertNotNull("The in-progress file should be recovered", bucket.getInProgressPart()); - assertEquals(0, bucket.getPendingFiles().size()); + + assertThat(bucket.getInProgressPart()) + .as("The in-progress file should be recovered") + .isNotNull(); + assertThat(bucket.getPendingFiles().size()).isEqualTo(0); } } @Test - public void testBucketIsRemovedWhenNotActive() throws Exception { - File outDir = TEMP_FOLDER.newFolder(); - Path path = new Path(outDir.toURI()); + public void testBucketIsRemovedWhenNotActive(@TempDir java.nio.file.Path tempDir) + throws Exception { + Path path = new Path(tempDir.toUri()); FileWriter fileWriter = createWriter( @@ -205,13 +206,12 @@ public void testBucketIsRemovedWhenNotActive() throws Exception { // No more records and another call to prepareCommit will makes it inactive fileWriter.prepareCommit(); - assertTrue(fileWriter.getActiveBuckets().isEmpty()); + assertThat(fileWriter.getActiveBuckets().isEmpty()).isTrue(); } @Test - public void testOnProcessingTime() throws Exception { - File outDir = TEMP_FOLDER.newFolder(); - Path path = new Path(outDir.toURI()); + public void testOnProcessingTime(@TempDir java.nio.file.Path tempDir) throws Exception { + Path path = new Path(tempDir.toUri()); // Create the processing timer service starts from 10. ManuallyTriggeredProcessingTimeService processingTimeService = @@ -237,15 +237,18 @@ public void testOnProcessingTime() throws Exception { processingTimeService.advanceTo(20); FileWriterBucket test1Bucket = fileWriter.getActiveBuckets().get("test1"); - assertNull( - "The in-progress part of test1 should be rolled", test1Bucket.getInProgressPart()); - assertEquals(1, test1Bucket.getPendingFiles().size()); + + assertThat(test1Bucket.getInProgressPart()) + .as("The in-progress part of test1 should be rolled") + .isNull(); + assertThat(test1Bucket.getPendingFiles().size()).isEqualTo(1); FileWriterBucket test2Bucket = fileWriter.getActiveBuckets().get("test2"); - assertNotNull( - "The in-progress part of test2 should not be rolled", - test2Bucket.getInProgressPart()); - assertEquals(0, test2Bucket.getPendingFiles().size()); + + assertThat(test2Bucket.getInProgressPart()) + .as("The in-progress part of test2 should not be rolled") + .isNotNull(); + assertThat(test2Bucket.getPendingFiles().size()).isEqualTo(0); // Close, pre-commit & clear all the pending records. processingTimeService.advanceTo(30); @@ -258,54 +261,68 @@ public void testOnProcessingTime() throws Exception { processingTimeService.advanceTo(40); test1Bucket = fileWriter.getActiveBuckets().get("test1"); - assertNull( - "The in-progress part of test1 should be rolled", test1Bucket.getInProgressPart()); - assertEquals(1, test1Bucket.getPendingFiles().size()); + + assertThat(test1Bucket.getInProgressPart()) + .as("The in-progress part of test1 should be rolled") + .isNull(); + assertThat(test1Bucket.getPendingFiles().size()).isEqualTo(1); test2Bucket = fileWriter.getActiveBuckets().get("test2"); - assertNotNull( - "The in-progress part of test2 should not be rolled", - test2Bucket.getInProgressPart()); - assertEquals(0, test2Bucket.getPendingFiles().size()); + + assertThat(test2Bucket.getInProgressPart()) + .as("The in-progress part of test2 should not be rolled") + .isNotNull(); + assertThat(test2Bucket.getPendingFiles().size()).isEqualTo(0); } @Test - public void testContextPassingNormalExecution() throws Exception { - testCorrectTimestampPassingInContext(1L, 2L, 3L); + public void testContextPassingNormalExecution(@TempDir java.nio.file.Path tempDir) + throws Exception { + testCorrectTimestampPassingInContext(1L, 2L, 3L, tempDir); } @Test - public void testContextPassingNullTimestamp() throws Exception { - testCorrectTimestampPassingInContext(null, 4L, 5L); + public void testContextPassingNullTimestamp(@TempDir java.nio.file.Path tempDir) + throws Exception { + testCorrectTimestampPassingInContext(null, 4L, 5L, tempDir); } @Test - public void testNumberRecordsOutCounter() throws IOException, InterruptedException { + public void testNumberRecordsOutCounter(@TempDir java.nio.file.Path tempDir) + throws IOException, InterruptedException { + Path path = new Path(tempDir.toUri()); + final OperatorIOMetricGroup operatorIOMetricGroup = UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup().getIOMetricGroup(); - File outDir = TEMP_FOLDER.newFolder(); - Path path = new Path(outDir.toURI()); - Counter recordsCounter = operatorIOMetricGroup.getNumRecordsOutCounter(); + final SinkWriterMetricGroup sinkWriterMetricGroup = + InternalSinkWriterMetricGroup.mock( + metricListener.getMetricGroup(), operatorIOMetricGroup); + + Counter recordsCounter = sinkWriterMetricGroup.getNumRecordsSendCounter(); SinkWriter.Context context = new ContextImpl(); FileWriter fileWriter = createWriter( path, DefaultRollingPolicy.builder().build(), new OutputFileConfig("part-", ""), - operatorIOMetricGroup); + sinkWriterMetricGroup); + + assertThat(recordsCounter.getCount()).isEqualTo(0); - assertEquals(0, recordsCounter.getCount()); fileWriter.write("1", context); - assertEquals(1, recordsCounter.getCount()); + + assertThat(recordsCounter.getCount()).isEqualTo(1); + fileWriter.write("2", context); fileWriter.write("3", context); - assertEquals(3, recordsCounter.getCount()); + + assertThat(recordsCounter.getCount()).isEqualTo(3); } private void testCorrectTimestampPassingInContext( - Long timestamp, long watermark, long processingTime) throws Exception { - final File outDir = TEMP_FOLDER.newFolder(); - final Path path = new Path(outDir.toURI()); + Long timestamp, long watermark, long processingTime, java.nio.file.Path tempDir) + throws Exception { + Path path = new Path(tempDir.toUri()); // Create the processing timer service starts from 10. ManuallyTriggeredProcessingTimeService processingTimeService = @@ -413,9 +430,9 @@ public String getBucketId(String element, BucketAssigner.Context context) { long watermark = context.currentWatermark(); long processingTime = context.currentProcessingTime(); - Assert.assertEquals(expectedTimestamp, elementTimestamp); - Assert.assertEquals(expectedProcessingTime, processingTime); - Assert.assertEquals(expectedWatermark, watermark); + assertThat(elementTimestamp).isEqualTo(expectedTimestamp); + assertThat(processingTime).isEqualTo(expectedProcessingTime); + assertThat(watermark).isEqualTo(expectedWatermark); return element; } @@ -432,13 +449,8 @@ private FileWriter createWriter( Path basePath, RollingPolicy rollingPolicy, OutputFileConfig outputFileConfig, - OperatorIOMetricGroup operatorIOMetricGroup) + SinkWriterMetricGroup sinkWriterMetricGroup) throws IOException { - final SinkWriterMetricGroup sinkWriterMetricGroup = - operatorIOMetricGroup == null - ? InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()) - : InternalSinkWriterMetricGroup.mock( - metricListener.getMetricGroup(), operatorIOMetricGroup); return new FileWriter<>( basePath, sinkWriterMetricGroup, @@ -458,7 +470,11 @@ private FileWriter createWriter( RollingPolicy rollingPolicy, OutputFileConfig outputFileConfig) throws IOException { - return createWriter(basePath, rollingPolicy, outputFileConfig, null); + return createWriter( + basePath, + rollingPolicy, + outputFileConfig, + InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup())); } private FileWriter createWriter( diff --git a/flink-connectors/flink-connector-gcp-pubsub/pom.xml b/flink-connectors/flink-connector-gcp-pubsub/pom.xml index f5409ec7eeda5..eb321155423c5 100644 --- a/flink-connectors/flink-connector-gcp-pubsub/pom.xml +++ b/flink-connectors/flink-connector-gcp-pubsub/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-hbase-1.4/pom.xml b/flink-connectors/flink-connector-hbase-1.4/pom.xml index 37ed9fee59752..4465ef287204c 100644 --- a/flink-connectors/flink-connector-hbase-1.4/pom.xml +++ b/flink-connectors/flink-connector-hbase-1.4/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-hbase-2.2/pom.xml b/flink-connectors/flink-connector-hbase-2.2/pom.xml index 0da174f1b5102..39a5fb5f2d45a 100644 --- a/flink-connectors/flink-connector-hbase-2.2/pom.xml +++ b/flink-connectors/flink-connector-hbase-2.2/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-hbase-base/pom.xml b/flink-connectors/flink-connector-hbase-base/pom.xml index 1de021f8573c6..338352b39d6f7 100644 --- a/flink-connectors/flink-connector-hbase-base/pom.xml +++ b/flink-connectors/flink-connector-hbase-base/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-hive/pom.xml b/flink-connectors/flink-connector-hive/pom.xml index 94728ee9f344e..a9cdbb33ab648 100644 --- a/flink-connectors/flink-connector-hive/pom.xml +++ b/flink-connectors/flink-connector-hive/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. @@ -549,13 +549,6 @@ under the License. test
- - org.apache.flink - flink-connector-base - ${project.version} - test - - org.apache.flink diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/ContinuousHiveSplitEnumerator.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/ContinuousHiveSplitEnumerator.java index 983faac860b14..a061b27d57609 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/ContinuousHiveSplitEnumerator.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/ContinuousHiveSplitEnumerator.java @@ -23,7 +23,6 @@ import org.apache.flink.api.connector.source.SplitEnumerator; import org.apache.flink.api.connector.source.SplitEnumeratorContext; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.connector.file.src.FileSourceSplit; import org.apache.flink.connector.file.src.PendingSplitsCheckpoint; import org.apache.flink.connector.file.src.assigners.FileSplitAssigner; @@ -78,7 +77,7 @@ public ContinuousHiveSplitEnumerator( Collection> seenPartitionsSinceOffset, FileSplitAssigner splitAssigner, long discoveryInterval, - ReadableConfig flinkConf, + int threadNum, JobConf jobConf, ObjectPath tablePath, ContinuousPartitionFetcher fetcher, @@ -95,7 +94,7 @@ public ContinuousHiveSplitEnumerator( currentReadOffset, seenPartitionsSinceOffset, tablePath, - flinkConf, + threadNum, jobConf, fetcher, fetcherContext); @@ -188,7 +187,7 @@ static class PartitionMonitor> private final Set> seenPartitionsSinceOffset; private final ObjectPath tablePath; - private final ReadableConfig flinkConf; + private final int threadNum; private final JobConf jobConf; private final ContinuousPartitionFetcher fetcher; private final HiveContinuousPartitionContext fetcherContext; @@ -197,14 +196,14 @@ static class PartitionMonitor> T currentReadOffset, Collection> seenPartitionsSinceOffset, ObjectPath tablePath, - ReadableConfig flinkConf, + int threadNum, JobConf jobConf, ContinuousPartitionFetcher fetcher, HiveContinuousPartitionContext fetcherContext) { this.currentReadOffset = currentReadOffset; this.seenPartitionsSinceOffset = new HashSet<>(seenPartitionsSinceOffset); this.tablePath = tablePath; - this.flinkConf = flinkConf; + this.threadNum = threadNum; this.jobConf = jobConf; this.fetcher = fetcher; this.fetcherContext = fetcherContext; @@ -244,7 +243,7 @@ public NewSplitsAndState call() throws Exception { 0, Collections.singletonList( fetcherContext.toHiveTablePartition(partition)), - flinkConf, + threadNum, jobConf)); } } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveLookupTableSource.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveLookupTableSource.java index 488ea1bc4c692..e68072affe0b6 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveLookupTableSource.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveLookupTableSource.java @@ -242,7 +242,7 @@ private TableFunction getLookupFunction(int[] keys) { PartitionReader partitionReader = new HiveInputFormatPartitionReader( - flinkConf, + flinkConf.get(HiveOptions.TABLE_EXEC_HIVE_LOAD_PARTITION_SPLITS_THREAD_NUM), jobConf, hiveVersion, tablePath, diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSource.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSource.java index 26c4ac59c0b33..d7f5209876194 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSource.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSource.java @@ -22,7 +22,6 @@ import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.SplitEnumerator; import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.connector.file.src.AbstractFileSource; import org.apache.flink.connector.file.src.ContinuousEnumerationSettings; import org.apache.flink.connector.file.src.PendingSplitsCheckpoint; @@ -60,7 +59,7 @@ public class HiveSource extends AbstractFileSource { private static final long serialVersionUID = 1L; - private final ReadableConfig flinkConf; + private final int threadNum; private final JobConfWrapper jobConfWrapper; private final List partitionKeys; private final ContinuousPartitionFetcher fetcher; @@ -73,7 +72,7 @@ public class HiveSource extends AbstractFileSource { FileSplitAssigner.Provider splitAssigner, BulkFormat readerFormat, @Nullable ContinuousEnumerationSettings continuousEnumerationSettings, - ReadableConfig flinkConf, + int threadNum, JobConf jobConf, ObjectPath tablePath, List partitionKeys, @@ -86,10 +85,10 @@ public class HiveSource extends AbstractFileSource { readerFormat, continuousEnumerationSettings); Preconditions.checkArgument( - flinkConf.get(HiveOptions.TABLE_EXEC_HIVE_LOAD_PARTITION_SPLITS_THREAD_NUM) >= 1, + threadNum >= 1, HiveOptions.TABLE_EXEC_HIVE_LOAD_PARTITION_SPLITS_THREAD_NUM.key() + " cannot be less than 1"); - this.flinkConf = flinkConf; + this.threadNum = threadNum; this.jobConfWrapper = new JobConfWrapper(jobConf); this.tablePath = tablePath; this.partitionKeys = partitionKeys; @@ -164,7 +163,7 @@ private boolean continuousPartitionedEnumerator() { seenPartitions, getAssignerFactory().create(new ArrayList<>(splits)), getContinuousEnumerationSettings().getDiscoveryInterval().toMillis(), - flinkConf, + threadNum, jobConfWrapper.conf(), tablePath, fetcher, diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceBuilder.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceBuilder.java index 25f7ed724afd1..75b6e7415332a 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceBuilder.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceBuilder.java @@ -68,6 +68,7 @@ import static org.apache.flink.connectors.hive.HiveOptions.STREAMING_SOURCE_MONITOR_INTERVAL; import static org.apache.flink.connectors.hive.HiveOptions.STREAMING_SOURCE_PARTITION_INCLUDE; import static org.apache.flink.connectors.hive.HiveOptions.STREAMING_SOURCE_PARTITION_ORDER; +import static org.apache.flink.connectors.hive.HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER; import static org.apache.flink.table.catalog.hive.util.HiveTableUtil.checkAcidTable; import static org.apache.flink.util.Preconditions.checkArgument; @@ -78,7 +79,8 @@ public class HiveSourceBuilder { private static final Duration DEFAULT_SCAN_MONITOR_INTERVAL = Duration.ofMinutes(1L); private final JobConf jobConf; - private final ReadableConfig flinkConf; + private final int threadNum; + private final boolean fallbackMappedReader; private final ObjectPath tablePath; private final Map tableOptions; @@ -110,7 +112,9 @@ public HiveSourceBuilder( @Nonnull String tableName, @Nonnull Map tableOptions) { this.jobConf = jobConf; - this.flinkConf = flinkConf; + this.threadNum = + flinkConf.get(HiveOptions.TABLE_EXEC_HIVE_LOAD_PARTITION_SPLITS_THREAD_NUM); + this.fallbackMappedReader = flinkConf.get(TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER); this.tablePath = new ObjectPath(dbName, tableName); this.hiveVersion = hiveVersion == null ? HiveShimLoader.getHiveVersion() : hiveVersion; HiveConf hiveConf = HiveConfUtils.create(jobConf); @@ -147,7 +151,9 @@ public HiveSourceBuilder( @Nullable String hiveVersion, @Nonnull CatalogTable catalogTable) { this.jobConf = jobConf; - this.flinkConf = flinkConf; + this.threadNum = + flinkConf.get(HiveOptions.TABLE_EXEC_HIVE_LOAD_PARTITION_SPLITS_THREAD_NUM); + this.fallbackMappedReader = flinkConf.get(TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER); this.tablePath = tablePath; this.hiveVersion = hiveVersion == null ? HiveShimLoader.getHiveVersion() : hiveVersion; this.fullSchema = catalogTable.getSchema(); @@ -231,12 +237,12 @@ public HiveSource buildWithBulkFormat(BulkFormat bulk new Path[1], new HiveSourceFileEnumerator.Provider( partitions != null ? partitions : Collections.emptyList(), - flinkConf, + threadNum, new JobConfWrapper(jobConf)), splitAssigner, bulkFormat, continuousSourceSettings, - flinkConf, + threadNum, jobConf, tablePath, partitionKeys, @@ -318,7 +324,7 @@ private BulkFormat createDefaultBulkFormat() { fullSchema.getFieldDataTypes(), hiveVersion, getProducedRowType(), - flinkConf.get(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER)), + fallbackMappedReader), limit); } } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceFileEnumerator.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceFileEnumerator.java index 4227c36824f19..dd0fe9807dbdc 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceFileEnumerator.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceFileEnumerator.java @@ -18,7 +18,6 @@ package org.apache.flink.connectors.hive; -import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.connector.file.src.FileSourceSplit; import org.apache.flink.connector.file.src.enumerate.FileEnumerator; import org.apache.flink.connectors.hive.read.HiveSourceSplit; @@ -45,33 +44,27 @@ public class HiveSourceFileEnumerator implements FileEnumerator { // For non-partition hive table, partitions only contains one partition which partitionValues is // empty. private final List partitions; - private final ReadableConfig flinkConf; + private final int threadNum; private final JobConf jobConf; public HiveSourceFileEnumerator( - List partitions, ReadableConfig flinkConf, JobConf jobConf) { + List partitions, int threadNum, JobConf jobConf) { this.partitions = partitions; - this.flinkConf = flinkConf; + this.threadNum = threadNum; this.jobConf = jobConf; } @Override public Collection enumerateSplits(Path[] paths, int minDesiredSplits) throws IOException { - return new ArrayList<>(createInputSplits(minDesiredSplits, partitions, flinkConf, jobConf)); + return new ArrayList<>(createInputSplits(minDesiredSplits, partitions, threadNum, jobConf)); } public static List createInputSplits( - int minNumSplits, - List partitions, - ReadableConfig flinkConf, - JobConf jobConf) + int minNumSplits, List partitions, int threadNum, JobConf jobConf) throws IOException { List hiveSplits = new ArrayList<>(); - try (MRSplitsGetter splitsGetter = - new MRSplitsGetter( - flinkConf.get( - HiveOptions.TABLE_EXEC_HIVE_LOAD_PARTITION_SPLITS_THREAD_NUM))) { + try (MRSplitsGetter splitsGetter = new MRSplitsGetter(threadNum)) { for (HiveTablePartitionSplits partitionSplits : splitsGetter.getHiveTablePartitionMRSplits(minNumSplits, partitions, jobConf)) { HiveTablePartition partition = partitionSplits.getHiveTablePartition(); @@ -109,21 +102,19 @@ public static class Provider implements FileEnumerator.Provider { private static final long serialVersionUID = 1L; private final List partitions; - private final ReadableConfig flinkConf; + private final int threadNum; private final JobConfWrapper jobConfWrapper; public Provider( - List partitions, - ReadableConfig flinkConf, - JobConfWrapper jobConfWrapper) { + List partitions, int threadNum, JobConfWrapper jobConfWrapper) { this.partitions = partitions; - this.flinkConf = flinkConf; + this.threadNum = threadNum; this.jobConfWrapper = jobConfWrapper; } @Override public FileEnumerator create() { - return new HiveSourceFileEnumerator(partitions, flinkConf, jobConfWrapper.conf()); + return new HiveSourceFileEnumerator(partitions, threadNum, jobConfWrapper.conf()); } } } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSink.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSink.java index 710c89039e7d2..431667092002d 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSink.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSink.java @@ -108,7 +108,8 @@ public class HiveTableSink implements DynamicTableSink, SupportsPartitioning, Su private static final Logger LOG = LoggerFactory.getLogger(HiveTableSink.class); - private final ReadableConfig flinkConf; + private final boolean fallbackMappedReader; + private final boolean fallbackMappedWriter; private final JobConf jobConf; private final CatalogTable catalogTable; private final ObjectIdentifier identifier; @@ -128,7 +129,24 @@ public HiveTableSink( ObjectIdentifier identifier, CatalogTable table, @Nullable Integer configuredParallelism) { - this.flinkConf = flinkConf; + this( + flinkConf.get(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER), + flinkConf.get(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_WRITER), + jobConf, + identifier, + table, + configuredParallelism); + } + + private HiveTableSink( + boolean fallbackMappedReader, + boolean fallbackMappedWriter, + JobConf jobConf, + ObjectIdentifier identifier, + CatalogTable table, + @Nullable Integer configuredParallelism) { + this.fallbackMappedReader = fallbackMappedReader; + this.fallbackMappedWriter = fallbackMappedWriter; this.jobConf = jobConf; this.identifier = identifier; this.catalogTable = table; @@ -298,7 +316,7 @@ private DataStreamSink createStreamSink( org.apache.flink.core.fs.Path path = new org.apache.flink.core.fs.Path(sd.getLocation()); BucketsBuilder> builder; - if (flinkConf.get(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_WRITER)) { + if (fallbackMappedWriter) { builder = bucketsBuilderForMRWriter( recordWriterFactory, sd, assigner, rollingPolicy, outputFileConfig); @@ -377,7 +395,7 @@ private CompactReader.Factory createCompactReaderFactory( catalogTable, hiveVersion, (RowType) tableSchema.toRowDataType().getLogicalType(), - flinkConf.get(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER)); + fallbackMappedReader); } private HiveTableMetaStoreFactory msFactory() { @@ -487,7 +505,12 @@ public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { public DynamicTableSink copy() { HiveTableSink sink = new HiveTableSink( - flinkConf, jobConf, identifier, catalogTable, configuredParallelism); + fallbackMappedReader, + fallbackMappedWriter, + jobConf, + identifier, + catalogTable, + configuredParallelism); sink.staticPartitionSpec = staticPartitionSpec; sink.overwrite = overwrite; sink.dynamicGrouping = dynamicGrouping; diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java index d7ee57850c856..cc5d63962c829 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java @@ -146,6 +146,8 @@ protected DataStream getDataStream( catalogTable.getPartitionKeys(), remainingPartitions); + int threadNum = + flinkConf.get(HiveOptions.TABLE_EXEC_HIVE_LOAD_PARTITION_SPLITS_THREAD_NUM); int parallelism = new HiveParallelismInference(tablePath, flinkConf) .infer( @@ -156,7 +158,7 @@ protected DataStream getDataStream( HiveSourceFileEnumerator.createInputSplits( 0, hivePartitionsToRead, - flinkConf, + threadNum, jobConf) .size()) .limit(limit); diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveInputFormatPartitionReader.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveInputFormatPartitionReader.java index 534c623b106df..2493942d8cdd9 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveInputFormatPartitionReader.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveInputFormatPartitionReader.java @@ -18,7 +18,6 @@ package org.apache.flink.connectors.hive.read; -import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.connector.file.table.PartitionReader; import org.apache.flink.connectors.hive.HiveTablePartition; import org.apache.flink.connectors.hive.JobConfWrapper; @@ -36,7 +35,7 @@ public class HiveInputFormatPartitionReader implements PartitionReader { private static final long serialVersionUID = 1L; - private final ReadableConfig flinkConf; + private final int threadNum; private final JobConfWrapper jobConfWrapper; private final String hiveVersion; protected final ObjectPath tablePath; @@ -51,7 +50,7 @@ public class HiveInputFormatPartitionReader private transient int readingSplitId; public HiveInputFormatPartitionReader( - ReadableConfig flinkConf, + int threadNum, JobConf jobConf, String hiveVersion, ObjectPath tablePath, @@ -60,7 +59,7 @@ public HiveInputFormatPartitionReader( List partitionKeys, int[] selectedFields, boolean useMapRedReader) { - this.flinkConf = flinkConf; + this.threadNum = threadNum; this.jobConfWrapper = new JobConfWrapper(jobConf); this.hiveVersion = hiveVersion; this.tablePath = tablePath; @@ -75,7 +74,7 @@ public HiveInputFormatPartitionReader( public void open(List partitions) throws IOException { hiveTableInputFormat = new HiveTableInputFormat( - this.flinkConf, + this.threadNum, this.jobConfWrapper.conf(), this.partitionKeys, this.fieldTypes, diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveTableInputFormat.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveTableInputFormat.java index 5297812bed5ff..eadd11417a91f 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveTableInputFormat.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveTableInputFormat.java @@ -23,8 +23,6 @@ import org.apache.flink.api.common.io.LocatableInputSplitAssigner; import org.apache.flink.api.common.io.statistics.BaseStatistics; import org.apache.flink.api.java.hadoop.common.HadoopInputFormatCommonBase; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.connectors.hive.HiveOptions; import org.apache.flink.connectors.hive.HiveTablePartition; import org.apache.flink.connectors.hive.HiveTablePartitionSplits; import org.apache.flink.connectors.hive.JobConfWrapper; @@ -69,7 +67,7 @@ public class HiveTableInputFormat extends HadoopInputFormatCommonBase partitionKeys, DataType[] fieldTypes, @@ -109,7 +107,7 @@ public HiveTableInputFormat( boolean useMapRedReader, List partitions) { super(jobConf.getCredentials()); - this.flinkConf = flinkConf; + this.threadNum = threadNum; this.jobConf = new JobConfWrapper(new JobConf(jobConf)); this.partitionKeys = partitionKeys; this.fieldTypes = fieldTypes; @@ -317,21 +315,15 @@ public void close() throws IOException { @Override public HiveTableInputSplit[] createInputSplits(int minNumSplits) throws IOException { - return createInputSplits(minNumSplits, partitions, flinkConf, jobConf.conf()); + return createInputSplits(minNumSplits, partitions, threadNum, jobConf.conf()); } public static HiveTableInputSplit[] createInputSplits( - int minNumSplits, - List partitions, - ReadableConfig flinkConf, - JobConf jobConf) + int minNumSplits, List partitions, int threadNum, JobConf jobConf) throws IOException { List hiveSplits = new ArrayList<>(); int splitNum = 0; - try (MRSplitsGetter splitsGetter = - new MRSplitsGetter( - flinkConf.get( - HiveOptions.TABLE_EXEC_HIVE_LOAD_PARTITION_SPLITS_THREAD_NUM))) { + try (MRSplitsGetter splitsGetter = new MRSplitsGetter(threadNum)) { for (HiveTablePartitionSplits partitionSplits : splitsGetter.getHiveTablePartitionMRSplits(minNumSplits, partitions, jobConf)) { for (InputSplit inputSplit : partitionSplits.getInputSplits()) { diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveASTParser.g b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveASTParser.g index a9fedc6b5041d..367d4b21cca2d 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveASTParser.g +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveASTParser.g @@ -732,8 +732,8 @@ catch (RecognitionException e) { // starting rule statement - : explainStatement EOF - | execStatement EOF + : explainStatement SEMICOLON? EOF + | execStatement SEMICOLON? EOF ; explainStatement 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 30537c20369f2..1aa6f506963cb 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 @@ -42,6 +42,7 @@ import org.apache.flink.table.operations.command.ResetOperation; import org.apache.flink.table.operations.command.SetOperation; import org.apache.flink.table.planner.delegation.hive.HiveParser; +import org.apache.flink.table.utils.CatalogManagerMocks; import org.apache.flink.types.Row; import org.apache.flink.util.CollectionUtil; import org.apache.flink.util.FileUtils; @@ -75,8 +76,6 @@ import java.util.LinkedHashMap; import java.util.List; -import static org.apache.flink.table.api.EnvironmentSettings.DEFAULT_BUILTIN_CATALOG; -import static org.apache.flink.table.api.EnvironmentSettings.DEFAULT_BUILTIN_DATABASE; import static org.hamcrest.CoreMatchers.instanceOf; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; @@ -670,17 +669,18 @@ public void testCatalog() { List catalogs = CollectionUtil.iteratorToList(tableEnv.executeSql("show catalogs").collect()); assertEquals(2, catalogs.size()); - tableEnv.executeSql("use catalog " + DEFAULT_BUILTIN_CATALOG); + tableEnv.executeSql("use catalog " + CatalogManagerMocks.DEFAULT_CATALOG); List databases = CollectionUtil.iteratorToList(tableEnv.executeSql("show databases").collect()); assertEquals(1, databases.size()); - assertEquals("+I[" + DEFAULT_BUILTIN_DATABASE + "]", databases.get(0).toString()); + assertEquals( + "+I[" + CatalogManagerMocks.DEFAULT_DATABASE + "]", databases.get(0).toString()); String catalogName = tableEnv.executeSql("show current catalog").collect().next().toString(); - assertEquals("+I[" + DEFAULT_BUILTIN_CATALOG + "]", catalogName); + assertEquals("+I[" + CatalogManagerMocks.DEFAULT_CATALOG + "]", catalogName); String databaseName = tableEnv.executeSql("show current database").collect().next().toString(); - assertEquals("+I[" + DEFAULT_BUILTIN_DATABASE + "]", databaseName); + assertEquals("+I[" + CatalogManagerMocks.DEFAULT_DATABASE + "]", databaseName); } @Test diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDynamicTableFactoryTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDynamicTableFactoryTest.java index 51659cd1b62a9..90e3bc72e7cc9 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDynamicTableFactoryTest.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDynamicTableFactoryTest.java @@ -290,7 +290,7 @@ private DynamicTableSource getTableSource(String tableName) throws Exception { hiveCatalog.getFactory().orElseThrow(IllegalStateException::new), tableIdentifier, tableEnvInternal.getCatalogManager().resolveCatalogTable(catalogTable), - tableEnv.getConfig().getConfiguration(), + tableEnv.getConfig(), Thread.currentThread().getContextClassLoader(), false); } @@ -306,7 +306,7 @@ private DynamicTableSink getTableSink(String tableName) throws Exception { hiveCatalog.getFactory().orElseThrow(IllegalStateException::new), tableIdentifier, tableEnvInternal.getCatalogManager().resolveCatalogTable(catalogTable), - tableEnv.getConfig().getConfiguration(), + tableEnv.getConfig(), Thread.currentThread().getContextClassLoader(), false); } 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 1dbb9a4d32b03..38834a123c3b3 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 @@ -408,7 +408,7 @@ private FileSystemLookupFunction getLookupFunction(String ta tableEnvInternal .getCatalogManager() .resolveCatalogTable(catalogTable), - tableEnv.getConfig().getConfiguration(), + tableEnv.getConfig(), Thread.currentThread().getContextClassLoader(), false); FileSystemLookupFunction lookupFunction = 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 fbf49e139cd17..606c351e31c85 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 @@ -256,9 +256,9 @@ public void testDecimal() throws Exception { TableEnvironment tableEnv = getTableEnvWithHiveCatalog(); tableEnv.executeSql("create database db1"); try { - tableEnv.executeSql("create table db1.src1 (x decimal(10,2))"); - tableEnv.executeSql("create table db1.src2 (x decimal(10,2))"); - tableEnv.executeSql("create table db1.dest (x decimal(10,2))"); + tableEnv.executeSql("create table db1.src1 (x decimal(12,2))"); + tableEnv.executeSql("create table db1.src2 (x decimal(12,2))"); + tableEnv.executeSql("create table db1.dest (x decimal(12,2))"); // populate src1 from Hive // TABLE keyword in INSERT INTO is mandatory prior to 1.1.0 hiveShell.execute( @@ -539,9 +539,7 @@ public void testOrcSchemaEvol() throws Exception { tableEnv.executeSql("create table db1.src (x smallint,y int) stored as orc"); hiveShell.execute("insert into table db1.src values (1,100),(2,200)"); - tableEnv.getConfig() - .getConfiguration() - .setBoolean(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER, true); + tableEnv.getConfig().set(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER, true); tableEnv.executeSql("alter table db1.src change x x int"); assertEquals( diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveSinkCompactionITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveSinkCompactionITCase.java index d2f6db56e55ee..ffe22221cac9f 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveSinkCompactionITCase.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveSinkCompactionITCase.java @@ -55,9 +55,7 @@ public void init() throws IOException { tEnv().useCatalog(hiveCatalog.getName()); // avoid too large parallelism lead to scheduler dead lock in streaming mode - tEnv().getConfig() - .getConfiguration() - .set(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM, false); + tEnv().getConfig().set(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM, false); super.init(); } 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 1d2b4f58fdaa1..39f7950acbc72 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 @@ -569,13 +569,9 @@ private void testStreamingWrite( tEnv.useCatalog(hiveCatalog.getName()); tEnv.getConfig().setSqlDialect(SqlDialect.HIVE); if (useMr) { - tEnv.getConfig() - .getConfiguration() - .set(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_WRITER, true); + tEnv.getConfig().set(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_WRITER, true); } else { - tEnv.getConfig() - .getConfiguration() - .set(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_WRITER, false); + tEnv.getConfig().set(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_WRITER, false); } try { 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 d9e1fadb2d7f5..df3023fdb74f6 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 @@ -473,9 +473,7 @@ public void testParallelismSettingWithFileNum() throws IOException { "select * from hive.source_db.test_parallelism_setting_with_file_num"); testParallelismSettingTranslateAndAssert(3, table, tEnv); - tEnv.getConfig() - .getConfiguration() - .setInteger(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM_MAX, 2); + tEnv.getConfig().set(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM_MAX, 2); testParallelismSettingTranslateAndAssert(2, table, tEnv); } @@ -496,12 +494,8 @@ public void testParallelismOnLimitPushDown() throws Exception { final String dbName = "source_db"; final String tblName = "test_parallelism_limit_pushdown"; TableEnvironment tEnv = createTableEnv(); - tEnv.getConfig() - .getConfiguration() - .setBoolean(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM, false); - tEnv.getConfig() - .getConfiguration() - .setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); + tEnv.getConfig().set(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM, false); + tEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); tEnv.executeSql( "CREATE TABLE source_db.test_parallelism_limit_pushdown " + "(`year` STRING, `value` INT) partitioned by (pt int)"); @@ -536,9 +530,7 @@ public void testParallelismWithoutParallelismInfer() throws Exception { tEnv.getConfig().setSqlDialect(SqlDialect.HIVE); tEnv.registerCatalog("hive", hiveCatalog); tEnv.useCatalog("hive"); - tEnv.getConfig() - .getConfiguration() - .setBoolean(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM, false); + tEnv.getConfig().set(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM, false); tEnv.executeSql( "CREATE TABLE source_db.test_parallelism_no_infer " + "(`year` STRING, `value` INT) partitioned by (pt int)"); @@ -773,9 +765,7 @@ private void testNonPartitionStreamingSource(Boolean useMapredReader, String tbl StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); StreamTableEnvironment tEnv = HiveTestUtils.createTableEnvInStreamingMode(env, SqlDialect.HIVE); - tEnv.getConfig() - .getConfiguration() - .setBoolean(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER, useMapredReader); + tEnv.getConfig().set(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER, useMapredReader); tEnv.registerCatalog(catalogName, hiveCatalog); tEnv.useCatalog(catalogName); tEnv.executeSql( @@ -838,15 +828,10 @@ private void testSourceConfig(boolean fallbackMR, boolean inferParallelism) thro doReturn(Optional.of(tableFactorySpy)).when(catalogSpy).getTableFactory(); TableEnvironment tableEnv = HiveTestUtils.createTableEnvInBatchMode(); + tableEnv.getConfig().set(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER, fallbackMR); tableEnv.getConfig() - .getConfiguration() - .setBoolean(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER, fallbackMR); - tableEnv.getConfig() - .getConfiguration() - .setBoolean(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM, inferParallelism); - tableEnv.getConfig() - .getConfiguration() - .setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); + .set(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM, inferParallelism); + tableEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); tableEnv.registerCatalog(catalogSpy.getName(), catalogSpy); tableEnv.useCatalog(catalogSpy.getName()); diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/PartitionMonitorTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/PartitionMonitorTest.java index f366f104c1551..0816092e9ae3b 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/PartitionMonitorTest.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/PartitionMonitorTest.java @@ -182,7 +182,8 @@ public List fetch(PartitionFetcher.Context context) 0L, seenPartitionsSinceOffset, tablePath, - configuration, + configuration.get( + HiveOptions.TABLE_EXEC_HIVE_LOAD_PARTITION_SPLITS_THREAD_NUM), jobConf, continuousPartitionFetcher, fetcherContext); 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 1d760d0653e97..349264d071d24 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 @@ -431,9 +431,7 @@ public void testParquetNameMapping() throws Exception { String.format( "create table db1.t2 (y int,x int) stored as parquet location '%s'", location)); - tableEnv.getConfig() - .getConfiguration() - .setBoolean(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER, true); + tableEnv.getConfig().set(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER, true); assertEquals( "[+I[1], +I[2]]", CollectionUtil.iteratorToList( @@ -584,9 +582,7 @@ public void testReadEmptyCollectionFromParquet() throws Exception { // test.parquet data: hehuiyuan {} [] String folderURI = this.getClass().getResource("/parquet").getPath(); - tableEnv.getConfig() - .getConfiguration() - .set(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER, true); + tableEnv.getConfig().set(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER, true); tableEnv.executeSql( String.format( "create external table src_t (a string, b map, c array) stored as %s location 'file://%s'", 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 bc8b97109ed33..aac9007cf107f 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 @@ -17,7 +17,7 @@ package org.apache.flink.connectors.hive.read; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.connectors.hive.HiveOptions; import org.apache.flink.connectors.hive.HiveTablePartition; import org.apache.flink.table.api.SqlDialect; import org.apache.flink.table.api.TableEnvironment; @@ -62,7 +62,7 @@ private void testReadFormat(TableEnvironment tableEnv, HiveCatalog hiveCatalog, // create partition reader HiveInputFormatPartitionReader partitionReader = new HiveInputFormatPartitionReader( - new Configuration(), + HiveOptions.TABLE_EXEC_HIVE_LOAD_PARTITION_SPLITS_THREAD_NUM.defaultValue(), new JobConf(hiveCatalog.getHiveConf()), hiveCatalog.getHiveVersion(), tablePath, 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 fe06b2bdd5830..77b289e166af2 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 @@ -41,6 +41,7 @@ import org.apache.flink.table.factories.TestManagedTableFactory; import org.apache.flink.table.planner.factories.utils.TestCollectionTableFactory; import org.apache.flink.table.types.AbstractDataType; +import org.apache.flink.table.utils.CatalogManagerMocks; import org.apache.flink.types.Row; import org.apache.flink.util.CollectionUtil; import org.apache.flink.util.FileUtils; @@ -217,9 +218,7 @@ public void testCsvTableViaAPI() throws Exception { public void testReadWriteCsv() throws Exception { // similar to CatalogTableITCase::testReadWriteCsvUsingDDL but uses HiveCatalog TableEnvironment tableEnv = TableEnvironment.create(EnvironmentSettings.inStreamingMode()); - tableEnv.getConfig() - .getConfiguration() - .setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1); + tableEnv.getConfig().set(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1); tableEnv.registerCatalog("myhive", hiveCatalog); tableEnv.useCatalog("myhive"); @@ -303,9 +302,7 @@ private TableEnvironment prepareTable(boolean isStreaming) { settings = EnvironmentSettings.inBatchMode(); } TableEnvironment tableEnv = TableEnvironment.create(settings); - tableEnv.getConfig() - .getConfiguration() - .setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1); + tableEnv.getConfig().set(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1); tableEnv.registerCatalog("myhive", hiveCatalog); tableEnv.useCatalog("myhive"); @@ -334,9 +331,7 @@ private TableEnvironment prepareTable(boolean isStreaming) { @Test public void testTableWithPrimaryKey() { TableEnvironment tableEnv = TableEnvironment.create(EnvironmentSettings.inStreamingMode()); - tableEnv.getConfig() - .getConfiguration() - .setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1); + tableEnv.getConfig().set(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1); tableEnv.registerCatalog("catalog1", hiveCatalog); tableEnv.useCatalog("catalog1"); @@ -385,7 +380,7 @@ public void testNewTableFactory() throws Exception { TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); tEnv.registerCatalog("myhive", hiveCatalog); tEnv.useCatalog("myhive"); - tEnv.getConfig().getConfiguration().set(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1); + tEnv.getConfig().set(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1); String path = this.getClass().getResource("/csv/test.csv").getPath(); @@ -472,15 +467,14 @@ public void testCreateTableLike() throws Exception { tableEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); tableEnv.useCatalog(hiveCatalog.getName()); tableEnv.executeSql("create table generic_table (x int) with ('connector'='COLLECTION')"); - tableEnv.useCatalog(EnvironmentSettings.DEFAULT_BUILTIN_CATALOG); + tableEnv.useCatalog(CatalogManagerMocks.DEFAULT_CATALOG); tableEnv.executeSql( String.format( "create table copy like `%s`.`default`.generic_table", hiveCatalog.getName())); - Catalog builtInCat = tableEnv.getCatalog(EnvironmentSettings.DEFAULT_BUILTIN_CATALOG).get(); + Catalog builtInCat = tableEnv.getCatalog(CatalogManagerMocks.DEFAULT_CATALOG).get(); CatalogBaseTable catalogTable = - builtInCat.getTable( - new ObjectPath(EnvironmentSettings.DEFAULT_BUILTIN_DATABASE, "copy")); + builtInCat.getTable(new ObjectPath(CatalogManagerMocks.DEFAULT_DATABASE, "copy")); assertThat(catalogTable.getOptions()).hasSize(1); assertThat(catalogTable.getOptions()) .containsEntry(FactoryUtil.CONNECTOR.key(), "COLLECTION"); 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 b3717ae11cc9c..4c8c5b75f2811 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 @@ -153,9 +153,7 @@ public static TableEnvironment createTableEnvInBatchMode() { public static TableEnvironment createTableEnvInBatchMode(SqlDialect dialect) { TableEnvironment tableEnv = TableEnvironment.create(EnvironmentSettings.inBatchMode()); - tableEnv.getConfig() - .getConfiguration() - .setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM.key(), 1); + tableEnv.getConfig().set(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1); tableEnv.getConfig().setSqlDialect(dialect); return tableEnv; } @@ -168,9 +166,7 @@ public static StreamTableEnvironment createTableEnvInStreamingMode( public static StreamTableEnvironment createTableEnvInStreamingMode( StreamExecutionEnvironment env, SqlDialect dialect) { StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env); - tableEnv.getConfig() - .getConfiguration() - .setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM.key(), 1); + tableEnv.getConfig().set(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1); tableEnv.getConfig().setSqlDialect(dialect); return tableEnv; } diff --git a/flink-connectors/flink-connector-jdbc/pom.xml b/flink-connectors/flink-connector-jdbc/pom.xml index aa3aa895c3f40..7a4116205fa2f 100644 --- a/flink-connectors/flink-connector-jdbc/pom.xml +++ b/flink-connectors/flink-connector-jdbc/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/MySqlCatalogITCase.java b/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/MySqlCatalogITCase.java index 96397923a7f28..0e2c48656e3e1 100644 --- a/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/MySqlCatalogITCase.java +++ b/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/MySqlCatalogITCase.java @@ -155,9 +155,7 @@ public class MySqlCatalogITCase extends MySqlCatalogTestBase { @Before public void setup() { this.tEnv = TableEnvironment.create(EnvironmentSettings.inStreamingMode()); - tEnv.getConfig() - .getConfiguration() - .setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM.key(), 1); + tEnv.getConfig().set(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1); // Use mysql catalog. tEnv.registerCatalog(TEST_CATALOG_NAME, catalog); diff --git a/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogITCase.java b/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogITCase.java index e6ec9cc2f5e7d..d85d8678b1ae4 100644 --- a/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogITCase.java +++ b/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogITCase.java @@ -40,9 +40,7 @@ public class PostgresCatalogITCase extends PostgresCatalogTestBase { @Before public void setup() { this.tEnv = TableEnvironment.create(EnvironmentSettings.inStreamingMode()); - tEnv.getConfig() - .getConfiguration() - .setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM.key(), 1); + tEnv.getConfig().set(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1); // use PG catalog tEnv.registerCatalog(TEST_CATALOG_NAME, catalog); @@ -114,7 +112,7 @@ public void testGroupByInsert() throws Exception { tEnv.executeSql( String.format( "insert into `%s` " - + "select `int`, cast('A' as bytes), `short`, max(`long`), max(`real`), " + + "select `int`, cast('41' as bytes), `short`, max(`long`), max(`real`), " + "max(`double_precision`), max(`numeric`), max(`decimal`), max(`boolean`), " + "max(`text`), 'B', 'C', max(`character_varying`), max(`timestamp`), " + "max(`date`), max(`time`), max(`default_numeric`) " @@ -128,7 +126,7 @@ public void testGroupByInsert() throws Exception { .execute() .collect()); assertEquals( - "[+I[1, [65], 3, 4, 5.5, 6.6, 7.70000, 8.8, true, a, B, C , d, 2016-06-22T19:10:25, 2015-01-01, 00:51:03, 500.000000000000000000]]", + "[+I[1, [52, 49], 3, 4, 5.5, 6.6, 7.70000, 8.8, true, a, B, C , d, 2016-06-22T19:10:25, 2015-01-01, 00:51:03, 500.000000000000000000]]", results.toString()); } diff --git a/flink-connectors/flink-connector-kafka/pom.xml b/flink-connectors/flink-connector-kafka/pom.xml index 96484a993d145..3799ebff9f8c7 100644 --- a/flink-connectors/flink-connector-kafka/pom.xml +++ b/flink-connectors/flink-connector-kafka/pom.xml @@ -26,7 +26,7 @@ under the License. flink-connectors org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. @@ -288,33 +288,6 @@ under the License. -Xms256m -Xmx2048m -Dmvn.forkNumber=${surefire.forkNumber} -XX:-UseGCOverheadLimit -Duser.country=US -Duser.language=en - - org.apache.maven.plugins - maven-shade-plugin - - - shade-flink - package - - shade - - - - - org.apache.flink:flink-connector-base - - - true - - - org.apache.flink.connector.base - org.apache.flink.connector.kafka.shaded.org.apache.flink.connector.base - - - - - - diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java index 71108d284add7..72690961e6b73 100644 --- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java +++ b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java @@ -20,6 +20,9 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.accumulators.ListAccumulator; +import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.common.eventtime.WatermarkGenerator; +import org.apache.flink.api.common.eventtime.WatermarkOutput; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -38,6 +41,7 @@ import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.ProcessFunction; import org.apache.flink.streaming.api.functions.sink.DiscardingSink; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.streaming.api.operators.StreamMap; @@ -70,11 +74,13 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; import static org.apache.flink.connector.kafka.testutils.KafkaSourceExternalContext.SplitMappingMode.PARTITION; import static org.apache.flink.connector.kafka.testutils.KafkaSourceExternalContext.SplitMappingMode.TOPIC; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; /** Unite test class for {@link KafkaSource}. */ @@ -256,6 +262,53 @@ public void testBasicReadWithoutGroupId(boolean enableObjectReuse) throws Except "testBasicReadWithoutGroupId"); executeAndVerify(env, stream); } + + @Test + public void testPerPartitionWatermark() throws Throwable { + String watermarkTopic = "watermarkTestTopic-" + UUID.randomUUID(); + KafkaSourceTestEnv.createTestTopic(watermarkTopic, 2, 1); + List> records = + Arrays.asList( + new ProducerRecord<>(watermarkTopic, 0, 100L, null, 100), + new ProducerRecord<>(watermarkTopic, 0, 200L, null, 200), + new ProducerRecord<>(watermarkTopic, 0, 300L, null, 300), + new ProducerRecord<>(watermarkTopic, 1, 150L, null, 150), + new ProducerRecord<>(watermarkTopic, 1, 250L, null, 250), + new ProducerRecord<>(watermarkTopic, 1, 350L, null, 350)); + KafkaSourceTestEnv.produceToKafka(records); + KafkaSource source = + KafkaSource.builder() + .setBootstrapServers(KafkaSourceTestEnv.brokerConnectionStrings) + .setTopics(watermarkTopic) + .setGroupId("watermark-test") + .setDeserializer(new TestingKafkaRecordDeserializationSchema(false)) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setBounded(OffsetsInitializer.latest()) + .build(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + env.fromSource( + source, + WatermarkStrategy.forGenerator( + (context) -> new OnEventWatermarkGenerator()), + "testPerPartitionWatermark") + .process( + new ProcessFunction() { + @Override + public void processElement( + PartitionAndValue value, + ProcessFunction.Context ctx, + Collector out) { + assertThat(ctx.timestamp()) + .as( + "Event time should never behind watermark " + + "because of per-split watermark multiplexing logic") + .isGreaterThanOrEqualTo( + ctx.timerService().currentWatermark()); + } + }); + env.execute(); + } } /** Integration test based on connector testing framework. */ @@ -400,4 +453,15 @@ public void invoke(PartitionAndValue value, Context context) { } }); } + + private static class OnEventWatermarkGenerator + implements WatermarkGenerator { + @Override + public void onEvent(PartitionAndValue event, long eventTimestamp, WatermarkOutput output) { + output.emitWatermark(new Watermark(eventTimestamp)); + } + + @Override + public void onPeriodicEmit(WatermarkOutput output) {} + } } diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java index 75791e58c5e8c..adfbdce6c9557 100644 --- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java +++ b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java @@ -78,6 +78,7 @@ public class KafkaTestEnvironmentImpl extends KafkaTestEnvironment { private final Map brokers = new HashMap<>(); private final Set pausedBroker = new HashSet<>(); private @Nullable GenericContainer zookeeper; + private @Nullable Network network; private String brokerConnectionString = ""; private Properties standardProps; private FlinkKafkaProducer.Semantic producerSemantic = FlinkKafkaProducer.Semantic.EXACTLY_ONCE; @@ -356,6 +357,10 @@ public void shutdown() throws Exception { if (zookeeper != null) { zookeeper.stop(); } + + if (network != null) { + network.close(); + } } private class KafkaOffsetHandlerImpl implements KafkaOffsetHandler { @@ -397,14 +402,14 @@ public void close() { } private void startKafkaContainerCluster(int numBrokers) { - Network network = Network.newNetwork(); if (numBrokers > 1) { + network = Network.newNetwork(); zookeeper = createZookeeperContainer(network); zookeeper.start(); LOG.info("Zookeeper container started"); } for (int brokerID = 0; brokerID < numBrokers; brokerID++) { - KafkaContainer broker = createKafkaContainer(network, brokerID, zookeeper); + KafkaContainer broker = createKafkaContainer(brokerID, zookeeper); brokers.put(brokerID, broker); } new ArrayList<>(brokers.values()).parallelStream().forEach(GenericContainer::start); @@ -426,11 +431,10 @@ private GenericContainer createZookeeperContainer(Network network) { } private KafkaContainer createKafkaContainer( - Network network, int brokerID, @Nullable GenericContainer zookeeper) { + int brokerID, @Nullable GenericContainer zookeeper) { String brokerName = String.format("Kafka-%d", brokerID); KafkaContainer broker = KafkaUtil.createKafkaContainer(DockerImageVersions.KAFKA, LOG, brokerName) - .withNetwork(network) .withNetworkAliases(brokerName) .withEnv("KAFKA_BROKER_ID", String.valueOf(brokerID)) .withEnv("KAFKA_MESSAGE_MAX_BYTES", String.valueOf(50 * 1024 * 1024)) @@ -447,6 +451,7 @@ private KafkaContainer createKafkaContainer( if (zookeeper != null) { broker.dependsOn(zookeeper) + .withNetwork(zookeeper.getNetwork()) .withExternalZookeeper( String.format("%s:%d", ZOOKEEPER_HOSTNAME, ZOOKEEPER_PORT)); } else { diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java index 62a7d6338d2f7..85c4af68e2d4b 100644 --- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java +++ b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java @@ -20,14 +20,16 @@ import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.common.serialization.SimpleStringSchema; -import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema; import org.apache.flink.connector.kafka.sink.KafkaSink; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.api.config.ExecutionConfigOptions; +import org.apache.flink.table.api.config.OptimizerConfigOptions; import org.apache.kafka.clients.producer.ProducerConfig; import org.junit.Before; @@ -58,11 +60,12 @@ public void testKafkaDebeziumChangelogSource() throws Exception { createTestTopic(topic, 1, 1); // enables MiniBatch processing to verify MiniBatch + FLIP-95, see FLINK-18769 - Configuration tableConf = tEnv.getConfig().getConfiguration(); - tableConf.setString("table.exec.mini-batch.enabled", "true"); - tableConf.setString("table.exec.mini-batch.allow-latency", "1s"); - tableConf.setString("table.exec.mini-batch.size", "5000"); - tableConf.setString("table.optimizer.agg-phase-strategy", "TWO_PHASE"); + TableConfig tableConf = tEnv.getConfig(); + tableConf.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true); + tableConf.set( + ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)); + tableConf.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, 5000L); + tableConf.set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE"); // ---------- Write the Debezium json into Kafka ------------------- List lines = readLines("debezium-data-schema-exclude.txt"); @@ -186,11 +189,12 @@ public void testKafkaCanalChangelogSource() throws Exception { // configure time zone of the Canal Json metadata "ingestion-timestamp" tEnv.getConfig().setLocalTimeZone(ZoneId.of("UTC")); // enables MiniBatch processing to verify MiniBatch + FLIP-95, see FLINK-18769 - Configuration tableConf = tEnv.getConfig().getConfiguration(); - tableConf.setString("table.exec.mini-batch.enabled", "true"); - tableConf.setString("table.exec.mini-batch.allow-latency", "1s"); - tableConf.setString("table.exec.mini-batch.size", "5000"); - tableConf.setString("table.optimizer.agg-phase-strategy", "TWO_PHASE"); + TableConfig tableConf = tEnv.getConfig(); + tableConf.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true); + tableConf.set( + ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)); + tableConf.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, 5000L); + tableConf.set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE"); // ---------- Write the Canal json into Kafka ------------------- List lines = readLines("canal-data.txt"); @@ -326,11 +330,12 @@ public void testKafkaMaxwellChangelogSource() throws Exception { // configure time zone of the Maxwell Json metadata "ingestion-timestamp" tEnv.getConfig().setLocalTimeZone(ZoneId.of("UTC")); // enables MiniBatch processing to verify MiniBatch + FLIP-95, see FLINK-18769 - Configuration tableConf = tEnv.getConfig().getConfiguration(); - tableConf.setString("table.exec.mini-batch.enabled", "true"); - tableConf.setString("table.exec.mini-batch.allow-latency", "1s"); - tableConf.setString("table.exec.mini-batch.size", "5000"); - tableConf.setString("table.optimizer.agg-phase-strategy", "TWO_PHASE"); + TableConfig tableConf = tEnv.getConfig(); + tableConf.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true); + tableConf.set( + ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)); + tableConf.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, 5000L); + tableConf.set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE"); // ---------- Write the Maxwell json into Kafka ------------------- List lines = readLines("maxwell-data.txt"); diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java index a6dab6da98843..cfd5a46a09e5e 100644 --- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java +++ b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java @@ -24,7 +24,9 @@ import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.utils.EncodingUtils; import org.apache.flink.test.util.SuccessException; import org.apache.flink.types.Row; @@ -44,7 +46,6 @@ import java.util.Collection; import java.util.Collections; import java.util.Comparator; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Random; @@ -58,6 +59,8 @@ import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.readLines; import static org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_SOURCE_IDLE_TIMEOUT; import static org.apache.flink.table.utils.TableTestMatchers.deepEqualTo; +import static org.apache.flink.util.CollectionUtil.entry; +import static org.apache.flink.util.CollectionUtil.map; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; @@ -299,29 +302,20 @@ public void testKafkaSourceSinkWithMetadata() throws Exception { + " %s\n" + ")", topic, bootstraps, groupId, formatOptions()); - tEnv.executeSql(createTable); String initialValues = "INSERT INTO kafka\n" + "VALUES\n" - + " ('data 1', 1, TIMESTAMP '2020-03-08 13:12:11.123', MAP['k1', X'C0FFEE', 'k2', X'BABE'], TRUE),\n" + + " ('data 1', 1, TIMESTAMP '2020-03-08 13:12:11.123', MAP['k1', X'C0FFEE', 'k2', X'BABE01'], TRUE),\n" + " ('data 2', 2, TIMESTAMP '2020-03-09 13:12:11.123', CAST(NULL AS MAP), FALSE),\n" - + " ('data 3', 3, TIMESTAMP '2020-03-10 13:12:11.123', MAP['k1', X'10', 'k2', X'20'], TRUE)"; + + " ('data 3', 3, TIMESTAMP '2020-03-10 13:12:11.123', MAP['k1', X'102030', 'k2', X'203040'], TRUE)"; tEnv.executeSql(initialValues).await(); // ---------- Consume stream from Kafka ------------------- final List result = collectRows(tEnv.sqlQuery("SELECT * FROM kafka"), 3); - final Map headers1 = new HashMap<>(); - headers1.put("k1", new byte[] {(byte) 0xC0, (byte) 0xFF, (byte) 0xEE}); - headers1.put("k2", new byte[] {(byte) 0xBA, (byte) 0xBE}); - - final Map headers3 = new HashMap<>(); - headers3.put("k1", new byte[] {(byte) 0x10}); - headers3.put("k2", new byte[] {(byte) 0x20}); - final List expected = Arrays.asList( Row.of( @@ -330,7 +324,9 @@ public void testKafkaSourceSinkWithMetadata() throws Exception { "CreateTime", LocalDateTime.parse("2020-03-08T13:12:11.123"), 0, - headers1, + map( + entry("k1", EncodingUtils.decodeHex("C0FFEE")), + entry("k2", EncodingUtils.decodeHex("BABE01"))), 0, topic, true), @@ -350,7 +346,9 @@ public void testKafkaSourceSinkWithMetadata() throws Exception { "CreateTime", LocalDateTime.parse("2020-03-10T13:12:11.123"), 0, - headers3, + map( + entry("k1", EncodingUtils.decodeHex("102030")), + entry("k2", EncodingUtils.decodeHex("203040"))), 0, topic, true)); @@ -527,7 +525,7 @@ public void testKafkaTemporalJoinChangelog() throws Exception { // 'value.source.timestamp'` DDL // will use the session time zone when convert the changelog time from milliseconds to // timestamp - tEnv.getConfig().getConfiguration().setString("table.local-time-zone", "UTC"); + tEnv.getConfig().set(TableConfigOptions.LOCAL_TIME_ZONE, "UTC"); // we always use a different topic name for each parameterized topic, // in order to make sure the topic can be created. @@ -763,9 +761,7 @@ public void testPerPartitionWatermarkWithIdleSource() throws Exception { // ---------- Produce an event time stream into Kafka ------------------- String groupId = getStandardProps().getProperty("group.id"); String bootstraps = getBootstrapServers(); - tEnv.getConfig() - .getConfiguration() - .set(TABLE_EXEC_SOURCE_IDLE_TIMEOUT, Duration.ofMillis(100)); + tEnv.getConfig().set(TABLE_EXEC_SOURCE_IDLE_TIMEOUT, Duration.ofMillis(100)); final String createTable = String.format( @@ -881,9 +877,7 @@ private TableResult startFromGroupOffset( // ---------- Produce an event time stream into Kafka ------------------- String bootstraps = getBootstrapServers(); - tEnv.getConfig() - .getConfiguration() - .set(TABLE_EXEC_SOURCE_IDLE_TIMEOUT, Duration.ofMillis(100)); + tEnv.getConfig().set(TABLE_EXEC_SOURCE_IDLE_TIMEOUT, Duration.ofMillis(100)); final String createTableSql = "CREATE TABLE %s (\n" diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java index d5810e2351687..20d03af74d297 100644 --- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java +++ b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java @@ -41,7 +41,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testcontainers.containers.KafkaContainer; -import org.testcontainers.containers.Network; import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.utility.DockerImageName; @@ -63,7 +62,6 @@ public abstract class KafkaTableTestBase extends AbstractTestBase { private static final Logger LOG = LoggerFactory.getLogger(KafkaTableTestBase.class); private static final String INTER_CONTAINER_KAFKA_ALIAS = "kafka"; - private static final Network NETWORK = Network.newNetwork(); private static final int zkTimeoutMills = 30000; @ClassRule @@ -77,7 +75,6 @@ protected void doStart() { } } }.withEmbeddedZookeeper() - .withNetwork(NETWORK) .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS) .withEnv( "KAFKA_TRANSACTION_MAX_TIMEOUT_MS", diff --git a/flink-connectors/flink-connector-kinesis/pom.xml b/flink-connectors/flink-connector-kinesis/pom.xml index e9cb2a5441e5d..8206b4f2acbdc 100644 --- a/flink-connectors/flink-connector-kinesis/pom.xml +++ b/flink-connectors/flink-connector-kinesis/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/NOTICE index a61e1eddbb234..375ffaf9aa84c 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/NOTICE @@ -54,8 +54,6 @@ This project bundles the following dependencies under the Apache Software Licens - io.netty:netty-transport-native-unix-common:4.1.70.Final - com.typesafe.netty:netty-reactive-streams-http:2.0.5 - com.typesafe.netty:netty-reactive-streams:2.0.5 -- commons-logging:commons-logging:1.1.3 -- com.fasterxml.jackson.core:jackson-core:2.13.0 This project bundles the following dependencies under the BSD license. See bundled license files for details. diff --git a/flink-connectors/flink-connector-nifi/pom.xml b/flink-connectors/flink-connector-nifi/pom.xml index 56b2a686ebba3..2873fb25eb9a6 100644 --- a/flink-connectors/flink-connector-nifi/pom.xml +++ b/flink-connectors/flink-connector-nifi/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-pulsar/archunit-violations/48c7dd05-c840-4ac4-a3ba-919e07191450 b/flink-connectors/flink-connector-pulsar/archunit-violations/48c7dd05-c840-4ac4-a3ba-919e07191450 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/flink-connectors/flink-connector-pulsar/archunit-violations/a2ce237e-b050-4ba0-8748-d83637a207a8 b/flink-connectors/flink-connector-pulsar/archunit-violations/a2ce237e-b050-4ba0-8748-d83637a207a8 new file mode 100644 index 0000000000000..36b34896247f5 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/archunit-violations/a2ce237e-b050-4ba0-8748-d83637a207a8 @@ -0,0 +1,6 @@ +org.apache.flink.connector.pulsar.sink.PulsarSinkITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule \ No newline at end of file diff --git a/flink-connectors/flink-connector-pulsar/archunit-violations/f4d91193-72ba-4ce4-ad83-98f780dce581 b/flink-connectors/flink-connector-pulsar/archunit-violations/f4d91193-72ba-4ce4-ad83-98f780dce581 new file mode 100644 index 0000000000000..7a71d25fc8140 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/archunit-violations/f4d91193-72ba-4ce4-ad83-98f780dce581 @@ -0,0 +1,12 @@ +org.apache.flink.connector.pulsar.sink.PulsarSinkITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.connector.pulsar.source.PulsarSourceITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule \ No newline at end of file diff --git a/flink-connectors/flink-connector-pulsar/archunit-violations/stored.rules b/flink-connectors/flink-connector-pulsar/archunit-violations/stored.rules new file mode 100644 index 0000000000000..ecdc39b0f1907 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/archunit-violations/stored.rules @@ -0,0 +1,4 @@ +# +#Thu May 12 11:35:25 CST 2022 +Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=48c7dd05-c840-4ac4-a3ba-919e07191450 +ITCASE\ tests\ should\ use\ a\ MiniCluster\ resource\ or\ extension=a2ce237e-b050-4ba0-8748-d83637a207a8 diff --git a/flink-connectors/flink-connector-pulsar/pom.xml b/flink-connectors/flink-connector-pulsar/pom.xml index 53e5ca400a0a0..d308c95451c6b 100644 --- a/flink-connectors/flink-connector-pulsar/pom.xml +++ b/flink-connectors/flink-connector-pulsar/pom.xml @@ -26,7 +26,7 @@ under the License. flink-connectors org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. @@ -36,18 +36,17 @@ under the License. jar - 2.9.1 + 2.10.0 0.6.1 + 1.7.0 3.11 - 3.6.3 - 4.1.72.Final - 1.33.0 + 4.1.74.Final + 1.42.1 - @@ -56,6 +55,34 @@ under the License. ${project.version} + + + org.apache.flink + flink-table-common + ${project.version} + provided + + + + org.apache.flink + flink-table-api-java-bridge + ${project.version} + provided + + + + org.apache.flink + flink-table-runtime + ${project.version} + provided + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${project.version} + provided + @@ -71,8 +98,22 @@ under the License. com.google.protobuf protobuf-java ${protoc.version} + + + + + + org.apache.flink + flink-json + ${project.version} + provided + + + + org.apache.flink + flink-avro + ${project.version} provided - true @@ -108,6 +149,50 @@ under the License. test + + + org.apache.flink + flink-table-common + ${project.version} + test-jar + test + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${project.version} + test + + + + org.apache.flink + flink-table-api-scala-bridge_${scala.binary.version} + ${project.version} + test + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${project.version} + test-jar + test + + + + + org.apache.flink + flink-avro-confluent-registry + ${project.version} + test + + + org.apache.flink + flink-csv + ${project.version} + test + + @@ -116,8 +201,28 @@ under the License. test - + + + org.apache.pulsar + testmocks + ${pulsar.version} + test + + + org.testng + testng + + + org.powermock + powermock-module-testng + + + org.apache.zookeeper + zookeeper + + + org.apache.pulsar pulsar-broker @@ -134,15 +239,6 @@ under the License. test - - - - org.apache.zookeeper - zookeeper - ${pulsar-zookeeper.version} - test - - @@ -171,10 +267,6 @@ under the License. javax.validation validation-api - - javax.xml.bind - jaxb-api - net.jcip jcip-annotations @@ -197,6 +289,20 @@ under the License. flink-architecture-tests-test test + + + + net.java.dev.jna + jna + 5.5.0 + test + + + net.java.dev.jna + jna-platform + 5.5.0 + test + @@ -227,7 +333,7 @@ under the License. kr.motd.maven os-maven-plugin - 1.7.0 + ${os-maven-plugin.version} @@ -236,7 +342,6 @@ under the License. org.apache.maven.plugins maven-surefire-plugin - true 1 -Xms256m -Xmx2048m -Dmvn.forkNumber=${surefire.forkNumber} @@ -340,32 +445,6 @@ under the License. - - org.apache.maven.plugins - maven-shade-plugin - - - shade-flink - package - - shade - - - - - org.apache.flink:flink-connector-base - - - - - org.apache.flink.connector.base - org.apache.flink.connector.pulsar.shaded.org.apache.flink.connector.base - - - - - - diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarClientFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarClientFactory.java index b1214b57c04f6..1f01b242143ca 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarClientFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarClientFactory.java @@ -153,7 +153,7 @@ public static PulsarClient createClient(PulsarConfiguration configuration) { /** * PulsarAdmin shares almost the same configuration with PulsarClient, but we separate this - * create method for directly creating it. + * creating method for directly use it. */ public static PulsarAdmin createAdmin(PulsarConfiguration configuration) { PulsarAdminBuilder builder = PulsarAdmin.builder(); @@ -200,15 +200,17 @@ private static Authentication createAuthentication(PulsarConfiguration configura String authParamsString = configuration.get(PULSAR_AUTH_PARAMS); return sneakyClient( () -> AuthenticationFactory.create(authPluginClassName, authParamsString)); - } else if (configuration.contains(PULSAR_AUTH_PARAM_MAP)) { - Map paramsMap = configuration.get(PULSAR_AUTH_PARAM_MAP); + } else { + Map paramsMap = configuration.getProperties(PULSAR_AUTH_PARAM_MAP); + if (paramsMap.isEmpty()) { + throw new IllegalArgumentException( + String.format( + "No %s or %s provided", + PULSAR_AUTH_PARAMS.key(), PULSAR_AUTH_PARAM_MAP.key())); + } + return sneakyClient( () -> AuthenticationFactory.create(authPluginClassName, paramsMap)); - } else { - throw new IllegalArgumentException( - String.format( - "No %s or %s provided", - PULSAR_AUTH_PARAMS.key(), PULSAR_AUTH_PARAM_MAP.key())); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarConfiguration.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarConfiguration.java index 3e64c6643cd62..0681a3ef4cba4 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarConfiguration.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarConfiguration.java @@ -76,7 +76,7 @@ public Map getProperties(ConfigOption> optio return properties; } - /** Get an option value from the given config, convert it into the a new value instance. */ + /** Get an option value from the given config, convert it into a new value instance. */ public T get(ConfigOption option, Function convertor) { F value = get(option); if (value != null) { diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarOptions.java index b06d9ed7c8abd..f878cb4efe099 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarOptions.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarOptions.java @@ -436,7 +436,7 @@ private PulsarOptions() { public static final ConfigOption PULSAR_MEMORY_LIMIT_BYTES = ConfigOptions.key(CLIENT_CONFIG_PREFIX + "memoryLimitBytes") .longType() - .defaultValue(0L) + .defaultValue(64 * 1024 * 1024L) .withDescription( Description.builder() .text( diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/BytesSchema.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/BytesSchema.java new file mode 100644 index 0000000000000..674c971b825f7 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/BytesSchema.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.common.schema; + +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.schema.SchemaInfo; + +import java.io.Serializable; +import java.nio.ByteBuffer; + +import static org.apache.pulsar.client.internal.PulsarClientImplementationBinding.getBytes; + +/** + * The schema is a wrapper for the original schema. It will send the schema info to Pulsar for + * compatibility check. And didn't deserialize messages. + */ +public class BytesSchema implements Schema, Serializable { + private static final long serialVersionUID = -539752264675729127L; + + private final PulsarSchema pulsarSchema; + + public BytesSchema(PulsarSchema pulsarSchema) { + this.pulsarSchema = pulsarSchema; + } + + @Override + public void validate(byte[] message) { + pulsarSchema.getPulsarSchema().validate(message); + } + + @Override + public byte[] encode(byte[] message) { + return message; + } + + @Override + public boolean supportSchemaVersioning() { + return pulsarSchema.getPulsarSchema().supportSchemaVersioning(); + } + + @Override + public byte[] decode(byte[] bytes) { + return bytes; + } + + @Override + public byte[] decode(byte[] bytes, byte[] schemaVersion) { + // None of Pulsar's schema implementations have implemented this method. + return bytes; + } + + @Override + public byte[] decode(ByteBuffer data, byte[] schemaVersion) { + return getBytes(data); + } + + @Override + public SchemaInfo getSchemaInfo() { + return pulsarSchema.getSchemaInfo(); + } + + @Override + public Schema clone() { + return this; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/PulsarSchema.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/PulsarSchema.java index 6ce91cdc67415..0d0d11497b793 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/PulsarSchema.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/PulsarSchema.java @@ -163,8 +163,7 @@ private void readObject(ObjectInputStream ois) throws ClassNotFoundException, IO // Schema int byteLen = ois.readInt(); byte[] schemaBytes = new byte[byteLen]; - int read = ois.read(schemaBytes); - checkState(read == byteLen); + ois.readFully(schemaBytes); // Type int typeIdx = ois.readInt(); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/AvroSchemaFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/AvroSchemaFactory.java index ab97b4102cc31..8288136288641 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/AvroSchemaFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/AvroSchemaFactory.java @@ -18,6 +18,9 @@ package org.apache.flink.connector.pulsar.common.schema.factories; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.AvroUtils; + import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.schema.SchemaDefinition; import org.apache.pulsar.client.impl.schema.AvroSchema; @@ -45,4 +48,14 @@ public Schema createSchema(SchemaInfo info) { return AvroSchema.of(definition); } + + @Override + public TypeInformation createTypeInfo(SchemaInfo info) { + try { + Class decodeClassInfo = decodeClassInfo(info); + return AvroUtils.getAvroUtils().createAvroTypeInfo(decodeClassInfo); + } catch (Exception e) { + return super.createTypeInfo(info); + } + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactory.java index 8d1a4944d0c5c..2280d353d95af 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactory.java @@ -18,6 +18,9 @@ package org.apache.flink.connector.pulsar.common.schema.factories; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.AvroUtils; + import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.schema.JSONSchema; import org.apache.pulsar.common.schema.SchemaInfo; @@ -38,4 +41,14 @@ public Schema createSchema(SchemaInfo info) { Class typeClass = decodeClassInfo(info); return JSONSchema.of(typeClass, info.getProperties()); } + + @Override + public TypeInformation createTypeInfo(SchemaInfo info) { + try { + Class decodeClassInfo = decodeClassInfo(info); + return AvroUtils.getAvroUtils().createAvroTypeInfo(decodeClassInfo); + } catch (Exception e) { + return super.createTypeInfo(info); + } + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/KeyValueSchemaFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/KeyValueSchemaFactory.java index 893e7e6eeb1db..d60f06d2a7643 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/KeyValueSchemaFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/KeyValueSchemaFactory.java @@ -69,11 +69,8 @@ public Schema> createSchema(SchemaInfo info) { public TypeInformation> createTypeInfo(SchemaInfo info) { KeyValue kvSchemaInfo = decodeKeyValueSchemaInfo(info); - Schema keySchema = PulsarSchemaUtils.createSchema(kvSchemaInfo.getKey()); - Class keyClass = decodeClassInfo(keySchema.getSchemaInfo()); - - Schema valueSchema = PulsarSchemaUtils.createSchema(kvSchemaInfo.getValue()); - Class valueClass = decodeClassInfo(valueSchema.getSchemaInfo()); + Class keyClass = decodeClassInfo(kvSchemaInfo.getKey()); + Class valueClass = decodeClassInfo(kvSchemaInfo.getValue()); Schema> schema = createSchema(info); PulsarSchema> pulsarSchema = diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/utils/PulsarSerdeUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/utils/PulsarSerdeUtils.java index 9f64172a504ef..93609bc720d69 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/utils/PulsarSerdeUtils.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/utils/PulsarSerdeUtils.java @@ -50,10 +50,7 @@ public static void serializeBytes(DataOutputStream out, byte[] bytes) throws IOE public static byte[] deserializeBytes(DataInputStream in) throws IOException { int size = in.readInt(); byte[] bytes = new byte[size]; - int result = in.read(bytes); - if (result < 0) { - throw new IOException("Couldn't deserialize the object, wrong byte buffer."); - } + in.readFully(bytes); return bytes; } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSink.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSink.java index 4c6c4a9b7c7d3..d952c5f016b32 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSink.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSink.java @@ -34,9 +34,13 @@ import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema; -import org.apache.flink.connector.pulsar.sink.writer.topic.TopicMetadataListener; +import org.apache.flink.connector.pulsar.sink.writer.topic.TopicRegister; import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.pulsar.client.api.CryptoKeyReader; + +import javax.annotation.Nullable; + import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -82,21 +86,25 @@ public class PulsarSink implements TwoPhaseCommittingSink serializationSchema; - private final TopicMetadataListener metadataListener; + private final TopicRegister topicRegister; private final MessageDelayer messageDelayer; private final TopicRouter topicRouter; + @Nullable private final CryptoKeyReader cryptoKeyReader; + PulsarSink( SinkConfiguration sinkConfiguration, PulsarSerializationSchema serializationSchema, - TopicMetadataListener metadataListener, + TopicRegister topicRegister, TopicRoutingMode topicRoutingMode, TopicRouter topicRouter, - MessageDelayer messageDelayer) { + MessageDelayer messageDelayer, + @Nullable CryptoKeyReader cryptoKeyReader) { this.sinkConfiguration = checkNotNull(sinkConfiguration); this.serializationSchema = checkNotNull(serializationSchema); - this.metadataListener = checkNotNull(metadataListener); + this.topicRegister = checkNotNull(topicRegister); this.messageDelayer = checkNotNull(messageDelayer); + this.cryptoKeyReader = cryptoKeyReader; checkNotNull(topicRoutingMode); // Create topic router supplier. @@ -125,9 +133,10 @@ public PrecommittingSinkWriter createWriter(InitContext i return new PulsarWriter<>( sinkConfiguration, serializationSchema, - metadataListener, + topicRegister, topicRouter, messageDelayer, + cryptoKeyReader, initContext); } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkBuilder.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkBuilder.java index 1668e3d126a16..2d886ef7947ad 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkBuilder.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkBuilder.java @@ -30,21 +30,33 @@ import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSchemaWrapper; import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema; -import org.apache.flink.connector.pulsar.sink.writer.topic.TopicMetadataListener; +import org.apache.flink.connector.pulsar.sink.writer.topic.TopicExtractor; +import org.apache.flink.connector.pulsar.sink.writer.topic.TopicRegister; +import org.apache.flink.connector.pulsar.sink.writer.topic.register.DynamicTopicRegister; +import org.apache.flink.connector.pulsar.sink.writer.topic.register.EmptyTopicRegister; +import org.apache.flink.connector.pulsar.sink.writer.topic.register.FixedTopicRegister; +import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.Schema; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + +import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.Properties; import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL; import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ENABLE_TRANSACTION; import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_SERVICE_URL; +import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_ENCRYPTION_KEYS; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_PRODUCER_NAME; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_SEND_TIMEOUT_MS; +import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_SINK_DEFAULT_TOPIC_PARTITIONS; +import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_SINK_TOPIC_AUTO_CREATION; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_WRITE_DELIVERY_GUARANTEE; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_WRITE_SCHEMA_EVOLUTION; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_WRITE_TRANSACTION_TIMEOUT; @@ -99,10 +111,12 @@ public class PulsarSinkBuilder { private final PulsarConfigBuilder configBuilder; private PulsarSerializationSchema serializationSchema; - private TopicMetadataListener metadataListener; + private TopicRegister topicRegister; private TopicRoutingMode topicRoutingMode; private TopicRouter topicRouter; private MessageDelayer messageDelayer; + @Nullable private CryptoKeyReader cryptoKeyReader; + private List encryptionKeys = new ArrayList<>(); // private builder constructor. PulsarSinkBuilder() { @@ -159,10 +173,26 @@ public PulsarSinkBuilder setTopics(String... topics) { * @return this PulsarSinkBuilder. */ public PulsarSinkBuilder setTopics(List topics) { - checkState(metadataListener == null, "setTopics couldn't be set twice."); + checkState(topicRegister == null, "setTopics couldn't be set twice."); // Making sure the topic should be distinct. List topicSet = distinctTopics(topics); - this.metadataListener = new TopicMetadataListener(topicSet); + if (topicSet.isEmpty()) { + this.topicRegister = new EmptyTopicRegister<>(); + } else { + this.topicRegister = new FixedTopicRegister<>(topicSet); + } + return this; + } + + /** + * Set a dynamic topic extractor for extracting the topic information. + * + * @return this PulsarSinkBuilder. + */ + public PulsarSinkBuilder setTopics(TopicExtractor extractor) { + checkState(topicRegister == null, "setTopics couldn't be set twice."); + this.topicRegister = new DynamicTopicRegister<>(extractor); + return this; } @@ -243,6 +273,56 @@ public PulsarSinkBuilder delaySendingMessage(MessageDelayer messageDelay return this; } + /** + * Sets a {@link CryptoKeyReader}. Configure the key reader to be used to encrypt the message + * payloads. + * + * @param cryptoKeyReader CryptoKeyReader object. + * @return this PulsarSinkBuilder. + */ + public PulsarSinkBuilder setCryptoKeyReader(CryptoKeyReader cryptoKeyReader) { + this.cryptoKeyReader = checkNotNull(cryptoKeyReader); + return this; + } + + /** + * Add public encryption key, used by producer to encrypt the data key. + * + *

At the time of producer creation, Pulsar client checks if there are keys added to + * encryptionKeys. If keys are found, a callback {@link CryptoKeyReader#getPrivateKey(String, + * Map)} and {@link CryptoKeyReader#getPublicKey(String, Map)} is invoked against each key to + * load the values of the key. Application should implement this callback to return the key in + * pkcs8 format. If compression is enabled, message is encrypted after compression. If batch + * messaging is enabled, the batched message is encrypted. + * + * @param keys Encryption keys. + * @return this PulsarSinkBuilder. + */ + public PulsarSinkBuilder setEncryptionKeys(String... keys) { + this.encryptionKeys.addAll(Arrays.asList(keys)); + return this; + } + + /** + * Pulsar sink disable the topic creation if the sink topic doesn't exist. You should explicitly + * set the default partition size for enabling topic creation. Make sure you have the authority + * on the given Pulsar admin token. + * + * @param partitionSize The partition size used on topic creation. It should be above to zero. + *

    + *
  • 0: we would create a non-partitioned topic. + *
  • above 0: we would create a partitioned topic with the given size. + *
+ * + * @return this PulsarSinkBuilder. + */ + public PulsarSinkBuilder enableTopicAutoCreation(int partitionSize) { + checkArgument(partitionSize >= 0); + configBuilder.set(PULSAR_SINK_TOPIC_AUTO_CREATION, true); + configBuilder.set(PULSAR_SINK_DEFAULT_TOPIC_PARTITIONS, partitionSize); + return this; + } + /** * Set an arbitrary property for the PulsarSink and Pulsar Producer. The valid keys can be found * in {@link PulsarSinkOptions} and {@link PulsarOptions}. @@ -315,6 +395,11 @@ public PulsarSink build() { if (!configBuilder.contains(PULSAR_PRODUCER_NAME)) { LOG.warn( "We recommend set a readable producer name through setProducerName(String) in production mode."); + } else { + String producerName = configBuilder.get(PULSAR_PRODUCER_NAME); + if (!producerName.contains("%s")) { + configBuilder.override(PULSAR_PRODUCER_NAME, producerName + " - %s"); + } } checkNotNull(serializationSchema, "serializationSchema must be set."); @@ -327,14 +412,14 @@ public PulsarSink build() { } // Topic metadata listener validation. - if (metadataListener == null) { + if (topicRegister == null) { if (topicRouter == null) { throw new NullPointerException( "No topic names or custom topic router are provided."); } else { LOG.warn( "No topic set has been provided, make sure your custom topic router support empty topic set."); - this.metadataListener = new TopicMetadataListener(); + this.topicRegister = new EmptyTopicRegister<>(); } } @@ -348,6 +433,13 @@ public PulsarSink build() { this.messageDelayer = MessageDelayer.never(); } + // Add the encryption keys if user provides one. + if (cryptoKeyReader != null) { + checkArgument( + !encryptionKeys.isEmpty(), "You should provide at least on encryption key."); + configBuilder.set(PULSAR_ENCRYPTION_KEYS, encryptionKeys); + } + // This is an unmodifiable configuration for Pulsar. // We don't use Pulsar's built-in configure classes for compatible requirement. SinkConfiguration sinkConfiguration = @@ -356,10 +448,11 @@ public PulsarSink build() { return new PulsarSink<>( sinkConfiguration, serializationSchema, - metadataListener, + topicRegister, topicRoutingMode, topicRouter, - messageDelayer); + messageDelayer, + cryptoKeyReader); } // ------------- private helpers -------------- diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkOptions.java index 3a7c5bc1a0e95..36aab9c8d051b 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkOptions.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkOptions.java @@ -29,20 +29,21 @@ import org.apache.flink.connector.pulsar.sink.writer.router.MessageKeyHash; import org.apache.pulsar.client.api.CompressionType; +import org.apache.pulsar.client.api.ProducerCryptoFailureAction; import java.time.Duration; +import java.util.List; import java.util.Map; import static java.util.Collections.emptyMap; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.apache.flink.configuration.description.LinkElement.link; import static org.apache.flink.configuration.description.TextElement.code; +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_MEMORY_LIMIT_BYTES; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PRODUCER_CONFIG_PREFIX; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.SINK_CONFIG_PREFIX; import static org.apache.flink.connector.pulsar.sink.writer.router.MessageKeyHash.MURMUR3_32_HASH; import static org.apache.pulsar.client.impl.conf.ProducerConfigurationData.DEFAULT_BATCHING_MAX_MESSAGES; -import static org.apache.pulsar.client.impl.conf.ProducerConfigurationData.DEFAULT_MAX_PENDING_MESSAGES; -import static org.apache.pulsar.client.impl.conf.ProducerConfigurationData.DEFAULT_MAX_PENDING_MESSAGES_ACROSS_PARTITIONS; /** * Configurations for PulsarSink. All the options list here could be configured in {@link @@ -128,6 +129,32 @@ private PulsarSinkOptions() { "The allowed transaction recommit times if we meet some retryable exception." + " This is used in Pulsar Transaction."); + /** @deprecated This config option was removed sink Flink 1.16 */ + @Deprecated + public static final ConfigOption PULSAR_MAX_PENDING_MESSAGES_ON_PARALLELISM = + ConfigOptions.key(SINK_CONFIG_PREFIX + "maxPendingMessages") + .intType() + .defaultValue(1000) + .withDescription( + Description.builder() + .text( + "The maximum number of pending messages in on sink parallelism.") + .build()); + + public static final ConfigOption PULSAR_SINK_TOPIC_AUTO_CREATION = + ConfigOptions.key(SINK_CONFIG_PREFIX + "topicAutoCreation") + .booleanType() + .defaultValue(false) + .withDescription( + "Enable the topic auto creation if the topic doesn't exist in Pulsar."); + + public static final ConfigOption PULSAR_SINK_DEFAULT_TOPIC_PARTITIONS = + ConfigOptions.key(SINK_CONFIG_PREFIX + "defaultTopicPartitions") + .intType() + .defaultValue(4) + .withDescription( + "If you enable the topic auto creation, you should also configure the default partition number here"); + /////////////////////////////////////////////////////////////////////////////// // // The configuration for ProducerConfigurationData part. @@ -155,10 +182,12 @@ private PulsarSinkOptions() { code("sendTimeout")) .build()); + /** @deprecated Use {@link PulsarOptions#PULSAR_MEMORY_LIMIT_BYTES} since Pulsar 2.10.0 */ + @Deprecated public static final ConfigOption PULSAR_MAX_PENDING_MESSAGES = ConfigOptions.key(PRODUCER_CONFIG_PREFIX + "maxPendingMessages") .intType() - .defaultValue(DEFAULT_MAX_PENDING_MESSAGES) + .noDefaultValue() .withDescription( Description.builder() .text("The maximum size of a queue holding pending messages.") @@ -174,12 +203,17 @@ private PulsarSinkOptions() { code("Send"), code("SendAsync"), code("BlockIfQueueFull")) + .text( + "Since Pulsar 2.10.0, you shouldn't set this option, use %s instead.", + code(PULSAR_MEMORY_LIMIT_BYTES.key())) .build()); + /** @deprecated Use {@link PulsarOptions#PULSAR_MEMORY_LIMIT_BYTES} since Pulsar 2.10.0 */ + @Deprecated public static final ConfigOption PULSAR_MAX_PENDING_MESSAGES_ACROSS_PARTITIONS = ConfigOptions.key(PRODUCER_CONFIG_PREFIX + "maxPendingMessagesAcrossPartitions") .intType() - .defaultValue(DEFAULT_MAX_PENDING_MESSAGES_ACROSS_PARTITIONS) + .noDefaultValue() .withDescription( Description.builder() .text( @@ -188,6 +222,9 @@ private PulsarSinkOptions() { .text( "Use the setting to lower the max pending messages for each partition (%s) if the total number exceeds the configured value.", code("setMaxPendingMessages")) + .text( + "Since Pulsar 2.10.0, you shouldn't set this option, use %s instead.", + code(PULSAR_MEMORY_LIMIT_BYTES.key())) .build()); public static final ConfigOption PULSAR_BATCHING_MAX_PUBLISH_DELAY_MICROS = @@ -266,4 +303,20 @@ private PulsarSinkOptions() { .text( " When getting a topic stats, associate this metadata with the consumer stats for easier identification.") .build()); + + public static final ConfigOption + PULSAR_PRODUCER_CRYPTO_FAILURE_ACTION = + ConfigOptions.key(PRODUCER_CONFIG_PREFIX + "producerCryptoFailureAction") + .enumType(ProducerCryptoFailureAction.class) + .defaultValue(ProducerCryptoFailureAction.FAIL) + .withDescription( + "The action the producer will take in case of encryption failures."); + + public static final ConfigOption> PULSAR_ENCRYPTION_KEYS = + ConfigOptions.key(PRODUCER_CONFIG_PREFIX + "encryptionKeys") + .stringType() + .asList() + .noDefaultValue() + .withDescription( + "Add public encryption key, used by producer to encrypt the data key."); } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/PulsarSinkConfigUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/PulsarSinkConfigUtils.java index 13821fe891852..f7f4ae89f9555 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/PulsarSinkConfigUtils.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/PulsarSinkConfigUtils.java @@ -27,12 +27,14 @@ import org.apache.pulsar.client.api.Schema; import java.util.Map; +import java.util.UUID; import static java.util.concurrent.TimeUnit.MICROSECONDS; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL; import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_AUTH_PARAMS; import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_AUTH_PARAM_MAP; +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_MEMORY_LIMIT_BYTES; import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_SERVICE_URL; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_BATCHING_ENABLED; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_BATCHING_MAX_BYTES; @@ -41,9 +43,11 @@ import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_BATCHING_PARTITION_SWITCH_FREQUENCY_BY_PUBLISH_DELAY; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_CHUNKING_ENABLED; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_COMPRESSION_TYPE; +import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_ENCRYPTION_KEYS; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_INITIAL_SEQUENCE_ID; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_MAX_PENDING_MESSAGES; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_MAX_PENDING_MESSAGES_ACROSS_PARTITIONS; +import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_PRODUCER_CRYPTO_FAILURE_ACTION; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_PRODUCER_NAME; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_PRODUCER_PROPERTIES; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_SEND_TIMEOUT_MS; @@ -63,6 +67,10 @@ private PulsarSinkConfigUtils() { .requiredOption(PULSAR_SERVICE_URL) .requiredOption(PULSAR_ADMIN_URL) .conflictOptions(PULSAR_AUTH_PARAMS, PULSAR_AUTH_PARAM_MAP) + .conflictOptions(PULSAR_MEMORY_LIMIT_BYTES, PULSAR_MAX_PENDING_MESSAGES) + .conflictOptions( + PULSAR_MEMORY_LIMIT_BYTES, + PULSAR_MAX_PENDING_MESSAGES_ACROSS_PARTITIONS) .build(); /** Create a pulsar producer builder by using the given Configuration. */ @@ -70,15 +78,14 @@ public static ProducerBuilder createProducerBuilder( PulsarClient client, Schema schema, SinkConfiguration configuration) { ProducerBuilder builder = client.newProducer(schema); - configuration.useOption(PULSAR_PRODUCER_NAME, builder::producerName); + configuration.useOption( + PULSAR_PRODUCER_NAME, + producerName -> String.format(producerName, UUID.randomUUID()), + builder::producerName); configuration.useOption( PULSAR_SEND_TIMEOUT_MS, Math::toIntExact, ms -> builder.sendTimeout(ms, MILLISECONDS)); - configuration.useOption(PULSAR_MAX_PENDING_MESSAGES, builder::maxPendingMessages); - configuration.useOption( - PULSAR_MAX_PENDING_MESSAGES_ACROSS_PARTITIONS, - builder::maxPendingMessagesAcrossPartitions); configuration.useOption( PULSAR_BATCHING_MAX_PUBLISH_DELAY_MICROS, s -> builder.batchingMaxPublishDelay(s, MICROSECONDS)); @@ -91,8 +98,17 @@ public static ProducerBuilder createProducerBuilder( configuration.useOption(PULSAR_CHUNKING_ENABLED, builder::enableChunking); configuration.useOption(PULSAR_COMPRESSION_TYPE, builder::compressionType); configuration.useOption(PULSAR_INITIAL_SEQUENCE_ID, builder::initialSequenceId); + configuration.useOption( + PULSAR_PRODUCER_CRYPTO_FAILURE_ACTION, builder::cryptoFailureAction); + + // Create the encryption keys. + if (configuration.contains(PULSAR_ENCRYPTION_KEYS)) { + for (String key : configuration.get(PULSAR_ENCRYPTION_KEYS)) { + builder.addEncryptionKey(key); + } + } - // Set producer properties + // Set producer properties. Map properties = configuration.getProperties(PULSAR_PRODUCER_PROPERTIES); if (!properties.isEmpty()) { builder.properties(properties); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/SinkConfiguration.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/SinkConfiguration.java index fe1204ee8791f..66c88e8a77b9d 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/SinkConfiguration.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/SinkConfiguration.java @@ -32,9 +32,10 @@ import java.util.Objects; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_BATCHING_MAX_MESSAGES; -import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_MAX_PENDING_MESSAGES_ACROSS_PARTITIONS; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_MAX_RECOMMIT_TIMES; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_MESSAGE_KEY_HASH; +import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_SINK_DEFAULT_TOPIC_PARTITIONS; +import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_SINK_TOPIC_AUTO_CREATION; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_TOPIC_METADATA_REFRESH_INTERVAL; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_WRITE_DELIVERY_GUARANTEE; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_WRITE_SCHEMA_EVOLUTION; @@ -51,8 +52,9 @@ public class SinkConfiguration extends PulsarConfiguration { private final int partitionSwitchSize; private final MessageKeyHash messageKeyHash; private final boolean enableSchemaEvolution; - private final int maxPendingMessages; private final int maxRecommitTimes; + private final boolean enableTopicAutoCreation; + private final int defaultTopicPartitions; public SinkConfiguration(Configuration configuration) { super(configuration); @@ -63,8 +65,9 @@ public SinkConfiguration(Configuration configuration) { this.partitionSwitchSize = getInteger(PULSAR_BATCHING_MAX_MESSAGES); this.messageKeyHash = get(PULSAR_MESSAGE_KEY_HASH); this.enableSchemaEvolution = get(PULSAR_WRITE_SCHEMA_EVOLUTION); - this.maxPendingMessages = get(PULSAR_MAX_PENDING_MESSAGES_ACROSS_PARTITIONS); this.maxRecommitTimes = get(PULSAR_MAX_RECOMMIT_TIMES); + this.enableTopicAutoCreation = get(PULSAR_SINK_TOPIC_AUTO_CREATION); + this.defaultTopicPartitions = get(PULSAR_SINK_DEFAULT_TOPIC_PARTITIONS); } /** The delivery guarantee changes the behavior of {@link PulsarWriter}. */ @@ -111,19 +114,21 @@ public boolean isEnableSchemaEvolution() { return enableSchemaEvolution; } - /** - * Pulsar message is sent asynchronously. Set this option for limiting the pending messages in a - * Pulsar writer instance. - */ - public int getMaxPendingMessages() { - return maxPendingMessages; - } - /** The maximum allowed recommitting time for a Pulsar transaction. */ public int getMaxRecommitTimes() { return maxRecommitTimes; } + /** Could the connector auto create the non-existed topics on the Pulsar? */ + public boolean isEnableTopicAutoCreation() { + return enableTopicAutoCreation; + } + + /** The default partition size when we enable the topic auto creation. */ + public int getDefaultTopicPartitions() { + return defaultTopicPartitions; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -136,25 +141,29 @@ public boolean equals(Object o) { return false; } SinkConfiguration that = (SinkConfiguration) o; - return transactionTimeoutMillis == that.transactionTimeoutMillis + return deliveryGuarantee == that.deliveryGuarantee + && transactionTimeoutMillis == that.transactionTimeoutMillis && topicMetadataRefreshInterval == that.topicMetadataRefreshInterval && partitionSwitchSize == that.partitionSwitchSize - && enableSchemaEvolution == that.enableSchemaEvolution && messageKeyHash == that.messageKeyHash - && maxPendingMessages == that.maxPendingMessages - && maxRecommitTimes == that.maxRecommitTimes; + && enableSchemaEvolution == that.enableSchemaEvolution + && maxRecommitTimes == that.maxRecommitTimes + && enableTopicAutoCreation == that.enableTopicAutoCreation + && defaultTopicPartitions == that.defaultTopicPartitions; } @Override public int hashCode() { return Objects.hash( super.hashCode(), + deliveryGuarantee, transactionTimeoutMillis, topicMetadataRefreshInterval, partitionSwitchSize, messageKeyHash, enableSchemaEvolution, - maxPendingMessages, - maxRecommitTimes); + maxRecommitTimes, + enableTopicAutoCreation, + defaultTopicPartitions); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java index 1e4113a8d03b1..fc5c73baa413b 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java @@ -19,7 +19,6 @@ package org.apache.flink.connector.pulsar.sink.writer; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.operators.MailboxExecutor; import org.apache.flink.api.common.operators.ProcessingTimeService; import org.apache.flink.api.common.serialization.SerializationSchema.InitializationContext; import org.apache.flink.api.connector.sink2.Sink.InitContext; @@ -33,22 +32,26 @@ import org.apache.flink.connector.pulsar.sink.writer.message.PulsarMessage; import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema; -import org.apache.flink.connector.pulsar.sink.writer.topic.TopicMetadataListener; -import org.apache.flink.connector.pulsar.sink.writer.topic.TopicProducerRegister; +import org.apache.flink.connector.pulsar.sink.writer.topic.ProducerRegister; +import org.apache.flink.connector.pulsar.sink.writer.topic.TopicRegister; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.shaded.guava30.com.google.common.base.Strings; +import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.TypedMessageBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.io.IOException; +import java.util.Base64; import java.util.Collection; import java.util.List; import java.util.Map; -import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicLong; import static java.util.Collections.emptyList; import static org.apache.flink.util.IOUtils.closeAll; @@ -64,17 +67,14 @@ public class PulsarWriter implements PrecommittingSinkWriter { private static final Logger LOG = LoggerFactory.getLogger(PulsarWriter.class); - private final SinkConfiguration sinkConfiguration; private final PulsarSerializationSchema serializationSchema; - private final TopicMetadataListener metadataListener; + private final TopicRegister topicRegister; private final TopicRouter topicRouter; private final MessageDelayer messageDelayer; private final DeliveryGuarantee deliveryGuarantee; private final PulsarSinkContext sinkContext; - private final MailboxExecutor mailboxExecutor; - private final TopicProducerRegister producerRegister; - - private long pendingMessages = 0; + private final ProducerRegister producerRegister; + private final AtomicLong pendingMessages = new AtomicLong(0); /** * Constructor creating a Pulsar writer. @@ -85,32 +85,32 @@ public class PulsarWriter implements PrecommittingSinkWriter serializationSchema, - TopicMetadataListener metadataListener, + TopicRegister topicRegister, TopicRouter topicRouter, MessageDelayer messageDelayer, + @Nullable CryptoKeyReader cryptoKeyReader, InitContext initContext) { - this.sinkConfiguration = checkNotNull(sinkConfiguration); + checkNotNull(sinkConfiguration); this.serializationSchema = checkNotNull(serializationSchema); - this.metadataListener = checkNotNull(metadataListener); + this.topicRegister = checkNotNull(topicRegister); this.topicRouter = checkNotNull(topicRouter); this.messageDelayer = checkNotNull(messageDelayer); checkNotNull(initContext); this.deliveryGuarantee = sinkConfiguration.getDeliveryGuarantee(); this.sinkContext = new PulsarSinkContextImpl(initContext, sinkConfiguration); - this.mailboxExecutor = initContext.getMailboxExecutor(); // Initialize topic metadata listener. LOG.debug("Initialize topic metadata after creating Pulsar writer."); ProcessingTimeService timeService = initContext.getProcessingTimeService(); - this.metadataListener.open(sinkConfiguration, timeService); + this.topicRegister.open(sinkConfiguration, timeService); // Initialize topic router. this.topicRouter.open(sinkConfiguration); @@ -125,7 +125,7 @@ public PulsarWriter( } // Create this producer register after opening serialization schema! - this.producerRegister = new TopicProducerRegister(sinkConfiguration); + this.producerRegister = new ProducerRegister(sinkConfiguration, cryptoKeyReader); } @Override @@ -133,9 +133,17 @@ public void write(IN element, Context context) throws IOException, InterruptedEx PulsarMessage message = serializationSchema.serialize(element, sinkContext); // Choose the right topic to send. - String key = message.getKey(); - List availableTopics = metadataListener.availableTopics(); - String topic = topicRouter.route(element, key, availableTopics, sinkContext); + List topics = topicRegister.topics(element); + + // TODO if both keyBytes and key are set, use keyBytes. This is a temporary solution. + String keyString; + if (message.getKeyBytes() == null) { + keyString = message.getKey(); + } else { + keyString = Base64.getEncoder().encodeToString(message.getKeyBytes()); + } + + String topic = topicRouter.route(element, keyString, topics, sinkContext); // Create message builder for sending message. TypedMessageBuilder builder = createMessageBuilder(topic, context, message); @@ -151,45 +159,26 @@ public void write(IN element, Context context) throws IOException, InterruptedEx // We would just ignore the sending exception. This may cause data loss. builder.sendAsync(); } else { - // Waiting for permits to write message. - requirePermits(); - mailboxExecutor.execute( - () -> enqueueMessageSending(topic, builder), - "Failed to send message to Pulsar"); - } - } - - private void enqueueMessageSending(String topic, TypedMessageBuilder builder) - throws ExecutionException, InterruptedException { - // Block the mailbox executor for yield method. - builder.sendAsync() - .whenComplete( - (id, ex) -> { - this.releasePermits(); - if (ex != null) { - throw new FlinkRuntimeException( - "Failed to send data to Pulsar " + topic, ex); - } else { - LOG.debug( - "Sent message to Pulsar {} with message id {}", topic, id); - } - }) - .get(); - } - - private void requirePermits() throws InterruptedException { - while (pendingMessages >= sinkConfiguration.getMaxPendingMessages()) { - LOG.info("Waiting for the available permits."); - mailboxExecutor.yield(); + // Increase the pending message count. + pendingMessages.incrementAndGet(); + builder.sendAsync() + .whenComplete( + (id, ex) -> { + pendingMessages.decrementAndGet(); + if (ex != null) { + throw new FlinkRuntimeException( + "Failed to send data to Pulsar " + topic, ex); + } else { + LOG.debug( + "Sent message to Pulsar {} with message id {}", + topic, + id); + } + }); } - pendingMessages++; - } - - private void releasePermits() { - this.pendingMessages -= 1; } - @SuppressWarnings("rawtypes") + @SuppressWarnings({"rawtypes", "unchecked"}) private TypedMessageBuilder createMessageBuilder( String topic, Context context, PulsarMessage message) { @@ -206,13 +195,18 @@ private TypedMessageBuilder createMessageBuilder( builder.key(key); } + byte[] keyBytes = message.getKeyBytes(); + if (keyBytes != null) { + builder.keyBytes(keyBytes); + } + long eventTime = message.getEventTime(); if (eventTime > 0) { builder.eventTime(eventTime); } else { // Set default message timestamp if flink has provided one. Long timestamp = context.timestamp(); - if (timestamp != null) { + if (timestamp != null && timestamp > 0) { builder.eventTime(timestamp); } } @@ -244,15 +238,15 @@ private TypedMessageBuilder createMessageBuilder( } @Override - public void flush(boolean endOfInput) throws IOException, InterruptedException { - if (endOfInput) { - // Try flush only once when we meet the end of the input. + public void flush(boolean endOfInput) throws IOException { + if (endOfInput || deliveryGuarantee != DeliveryGuarantee.NONE) { + LOG.info("Flush the pending messages to Pulsar."); + + // Try to flush pending messages. producerRegister.flush(); - } else { - while (pendingMessages != 0 && deliveryGuarantee != DeliveryGuarantee.NONE) { + // Make sure all the pending messages should be flushed to Pulsar. + while (pendingMessages.longValue() > 0) { producerRegister.flush(); - LOG.info("Flush the pending messages to Pulsar."); - mailboxExecutor.yield(); } } } @@ -269,6 +263,6 @@ public Collection prepareCommit() { @Override public void close() throws Exception { // Close all the resources and throw the exception at last. - closeAll(metadataListener, producerRegister); + closeAll(topicRegister, producerRegister); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessage.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessage.java index 0c45763cdb1ef..0dd52655f9040 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessage.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessage.java @@ -37,6 +37,7 @@ public class PulsarMessage { @Nullable private final byte[] orderingKey; @Nullable private final String key; + @Nullable private final byte[] keyBytes; private final long eventTime; private final Schema schema; @Nullable private final T value; @@ -49,6 +50,7 @@ public class PulsarMessage { PulsarMessage( @Nullable byte[] orderingKey, @Nullable String key, + @Nullable byte[] keyBytes, long eventTime, Schema schema, @Nullable T value, @@ -58,6 +60,7 @@ public class PulsarMessage { boolean disableReplication) { this.orderingKey = orderingKey; this.key = key; + this.keyBytes = keyBytes; this.eventTime = eventTime; this.schema = schema; this.value = value; @@ -77,6 +80,11 @@ public String getKey() { return key; } + @Nullable + public byte[] getKeyBytes() { + return keyBytes; + } + public long getEventTime() { return eventTime; } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessageBuilder.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessageBuilder.java index 9330d09a3d03b..fe4cef02964fa 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessageBuilder.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessageBuilder.java @@ -36,6 +36,7 @@ public class PulsarMessageBuilder { private byte[] orderingKey; private String key; + private byte[] keyBytes; private long eventTime; Schema schema; private T value; @@ -56,7 +57,16 @@ public PulsarMessageBuilder orderingKey(byte[] orderingKey) { */ public PulsarMessageBuilder key(String key) { this.key = checkNotNull(key); - return null; + return this; + } + + /** + * Property {@link TypedMessageBuilder#keyBytes(byte[])}. This property would also be used in + * {@link KeyHashTopicRouter}. + */ + public PulsarMessageBuilder keyBytes(byte[] keyBytes) { + this.keyBytes = checkNotNull(keyBytes); + return this; } /** Method wrapper of {@link TypedMessageBuilder#eventTime(long)}. */ @@ -116,6 +126,7 @@ public PulsarMessage build() { return new PulsarMessage<>( orderingKey, key, + keyBytes, eventTime, schema, value, diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/router/MessageKeyHash.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/router/MessageKeyHash.java index 7f35760f2531d..bbac99e4b961d 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/router/MessageKeyHash.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/router/MessageKeyHash.java @@ -25,7 +25,7 @@ import org.apache.pulsar.client.impl.Hash; import org.apache.pulsar.client.impl.JavaStringHash; -import org.apache.pulsar.client.impl.Murmur3_32Hash; +import org.apache.pulsar.client.impl.Murmur3Hash32; import static org.apache.flink.configuration.description.LinkElement.link; import static org.apache.flink.configuration.description.TextElement.code; @@ -57,7 +57,7 @@ public Hash getHash() { link("https://en.wikipedia.org/wiki/MurmurHash", "Murmur3"))) { @Override public Hash getHash() { - return Murmur3_32Hash.getInstance(); + return Murmur3Hash32.getInstance(); } }; diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicProducerRegister.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/ProducerRegister.java similarity index 90% rename from flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicProducerRegister.java rename to flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/ProducerRegister.java index 9bb175319d2a1..d25b8ef00521b 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicProducerRegister.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/ProducerRegister.java @@ -26,6 +26,7 @@ import org.apache.flink.shaded.guava30.com.google.common.io.Closer; +import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerBuilder; import org.apache.pulsar.client.api.PulsarClient; @@ -37,6 +38,8 @@ import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.schema.SchemaInfo; +import javax.annotation.Nullable; + import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; @@ -56,17 +59,20 @@ * we have to create different instances for different topics. */ @Internal -public class TopicProducerRegister implements Closeable { +public class ProducerRegister implements Closeable { private final PulsarClient pulsarClient; private final SinkConfiguration sinkConfiguration; - private final Map>> producerRegister; + @Nullable private final CryptoKeyReader cryptoKeyReader; + private final Map>> register; private final Map transactionRegister; - public TopicProducerRegister(SinkConfiguration sinkConfiguration) { + public ProducerRegister( + SinkConfiguration sinkConfiguration, @Nullable CryptoKeyReader cryptoKeyReader) { this.pulsarClient = createClient(sinkConfiguration); this.sinkConfiguration = sinkConfiguration; - this.producerRegister = new HashMap<>(); + this.cryptoKeyReader = cryptoKeyReader; + this.register = new HashMap<>(); this.transactionRegister = new HashMap<>(); } @@ -110,7 +116,7 @@ public List prepareCommit() { * successfully persisted. */ public void flush() throws IOException { - Collection>> collection = producerRegister.values(); + Collection>> collection = register.values(); for (Map> producers : collection) { for (Producer producer : producers.values()) { producer.flush(); @@ -128,7 +134,7 @@ public void close() throws IOException { closer.register(this::abortTransactions); // Remove all the producers. - closer.register(producerRegister::clear); + closer.register(register::clear); // All the producers would be closed by this method. // We would block until all the producers have been successfully closed. @@ -140,7 +146,7 @@ public void close() throws IOException { @SuppressWarnings("unchecked") private Producer getOrCreateProducer(String topic, Schema schema) { Map> producers = - producerRegister.computeIfAbsent(topic, key -> new HashMap<>()); + register.computeIfAbsent(topic, key -> new HashMap<>()); SchemaInfo schemaInfo = schema.getSchemaInfo(); if (producers.containsKey(schemaInfo)) { @@ -148,6 +154,12 @@ private Producer getOrCreateProducer(String topic, Schema schema) { } else { ProducerBuilder builder = createProducerBuilder(pulsarClient, schema, sinkConfiguration); + + // Set the message crypto key reader. + if (cryptoKeyReader != null) { + builder.cryptoKeyReader(cryptoKeyReader); + } + // Set the required topic name. builder.topic(topic); Producer producer = sneakyClient(builder::create); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicExtractor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicExtractor.java new file mode 100644 index 0000000000000..853698220f46d --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicExtractor.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.sink.writer.topic; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; + +import org.apache.pulsar.client.admin.PulsarAdmin; + +import java.io.Serializable; + +/** Choose topics from the message, used for dynamically generate topics in Pulsar sink. */ +@PublicEvolving +public interface TopicExtractor extends Serializable { + + /** + * @param in The message would be written to Pulsar. + * @param provider Used for query topic metadata. + * @return The topic you want to use. You can use both partitioned topic name or a topic name + * without partition information. We would query the partition information and pass it to + * {@link TopicRouter} if you return a topic name without partition information. + */ + TopicPartition extract(IN in, TopicMetadataProvider provider); + + /** Implement this method if you have some non-serializable field. */ + default void open(SinkConfiguration sinkConfiguration) { + // Nothing to do by default. + } + + /** + * A wrapper for {@link PulsarAdmin} instance, we won't expose the Pulsar admin interface for + * better control the abstraction and cache support. + */ + @PublicEvolving + interface TopicMetadataProvider { + + /** + * The query result from Pulsar would be cached, if you set positive value in {@link + * SinkConfiguration#getTopicMetadataRefreshInterval()}. + * + * @throws Exception Failed to query Pulsar metadata would throw this exception. + */ + TopicMetadata query(String topic) throws Exception; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicRegister.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicRegister.java new file mode 100644 index 0000000000000..925334923e72c --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicRegister.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.sink.writer.topic; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.operators.ProcessingTimeService; +import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; + +import java.io.Closeable; +import java.io.Serializable; +import java.util.List; + +/** The topic register for returning the available topic partitions. */ +@Internal +public interface TopicRegister extends Serializable, Closeable { + + /** + * Return all the available topic partitions. We would recalculate the partitions if the topic + * metadata has been changed. Otherwise, we would return the cached result for better + * performance. + */ + List topics(IN in); + + /** Register the topic metadata update in process time service. */ + void open(SinkConfiguration sinkConfiguration, ProcessingTimeService timeService); +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/CachedTopicMetadataProvider.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/CachedTopicMetadataProvider.java new file mode 100644 index 0000000000000..5d3a3729f2d54 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/CachedTopicMetadataProvider.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.sink.writer.topic.metadata; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; +import org.apache.flink.connector.pulsar.sink.writer.topic.TopicExtractor.TopicMetadataProvider; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata; + +import org.apache.flink.shaded.guava30.com.google.common.cache.Cache; +import org.apache.flink.shaded.guava30.com.google.common.cache.CacheBuilder; + +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; + +import java.util.concurrent.TimeUnit; + +/** + * The default implementation for querying topic metadata. The query result would be cached by + * positive {@link SinkConfiguration#getTopicMetadataRefreshInterval()}. + */ +@Internal +public class CachedTopicMetadataProvider implements TopicMetadataProvider { + + private final PulsarAdmin pulsarAdmin; + private final Cache metadataCache; + + public CachedTopicMetadataProvider( + PulsarAdmin pulsarAdmin, SinkConfiguration sinkConfiguration) { + this.pulsarAdmin = pulsarAdmin; + + long refreshInterval = sinkConfiguration.getTopicMetadataRefreshInterval(); + if (refreshInterval <= 0) { + // Disable cache expires, the query result will never be kept in the cache. + this.metadataCache = null; + } else { + this.metadataCache = + CacheBuilder.newBuilder() + .expireAfterWrite(refreshInterval, TimeUnit.MILLISECONDS) + .maximumSize(1000) + .build(); + } + } + + @Override + public TopicMetadata query(String topic) throws PulsarAdminException { + TopicMetadata metadata = metadataCache == null ? null : metadataCache.getIfPresent(topic); + + if (metadata == null) { + PartitionedTopicMetadata meta = pulsarAdmin.topics().getPartitionedTopicMetadata(topic); + metadata = new TopicMetadata(topic, meta.partitions); + if (metadataCache != null) { + metadataCache.put(topic, metadata); + } + } + + return metadata; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/NotExistedTopicMetadataProvider.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/NotExistedTopicMetadataProvider.java new file mode 100644 index 0000000000000..15926916ed365 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/NotExistedTopicMetadataProvider.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.sink.writer.topic.metadata; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; +import org.apache.flink.connector.pulsar.sink.writer.topic.TopicExtractor.TopicMetadataProvider; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata; + +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.admin.PulsarAdminException.NotFoundException; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; + +import static org.apache.flink.util.ExceptionUtils.findThrowable; +import static org.apache.pulsar.common.partition.PartitionedTopicMetadata.NON_PARTITIONED; + +/** + * Shared topic metadata query for {@code TopicRegister}. We would auto create the topics if you + * enable this feature. + */ +@Internal +public class NotExistedTopicMetadataProvider implements TopicMetadataProvider { + + private final PulsarAdmin pulsarAdmin; + private final boolean enableTopicAutoCreation; + private final int defaultTopicPartitions; + + public NotExistedTopicMetadataProvider( + PulsarAdmin pulsarAdmin, SinkConfiguration sinkConfiguration) { + this.pulsarAdmin = pulsarAdmin; + this.enableTopicAutoCreation = sinkConfiguration.isEnableTopicAutoCreation(); + this.defaultTopicPartitions = sinkConfiguration.getDefaultTopicPartitions(); + } + + @Override + public TopicMetadata query(String topic) throws PulsarAdminException { + try { + PartitionedTopicMetadata meta = pulsarAdmin.topics().getPartitionedTopicMetadata(topic); + return new TopicMetadata(topic, meta.partitions); + } catch (PulsarAdminException e) { + if (findThrowable(e, NotFoundException.class).isPresent() && enableTopicAutoCreation) { + createTopic(topic); + return new TopicMetadata(topic, defaultTopicPartitions); + } else { + throw e; + } + } + } + + private void createTopic(String topic) throws PulsarAdminException { + if (defaultTopicPartitions == NON_PARTITIONED) { + pulsarAdmin.topics().createNonPartitionedTopic(topic); + } else { + pulsarAdmin.topics().createPartitionedTopic(topic, defaultTopicPartitions); + } + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/DynamicTopicRegister.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/DynamicTopicRegister.java new file mode 100644 index 0000000000000..7dea2a159d4d3 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/DynamicTopicRegister.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.sink.writer.topic.register; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.operators.ProcessingTimeService; +import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; +import org.apache.flink.connector.pulsar.sink.writer.topic.TopicExtractor; +import org.apache.flink.connector.pulsar.sink.writer.topic.TopicRegister; +import org.apache.flink.connector.pulsar.sink.writer.topic.metadata.CachedTopicMetadataProvider; +import org.apache.flink.connector.pulsar.sink.writer.topic.metadata.NotExistedTopicMetadataProvider; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; +import org.apache.flink.util.FlinkRuntimeException; + +import org.apache.flink.shaded.guava30.com.google.common.cache.Cache; +import org.apache.flink.shaded.guava30.com.google.common.cache.CacheBuilder; + +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import static java.util.Collections.singletonList; +import static org.apache.flink.connector.pulsar.common.config.PulsarClientFactory.createAdmin; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** The register for returning dynamic topic partitions information. */ +@Internal +public class DynamicTopicRegister implements TopicRegister { + private static final long serialVersionUID = 4374769306761301456L; + + private final TopicExtractor topicExtractor; + + // Dynamic fields. + private transient PulsarAdmin pulsarAdmin; + private transient CachedTopicMetadataProvider cachedMetadataProvider; + private transient NotExistedTopicMetadataProvider notExistedMetadataProvider; + private transient Cache> partitionsCache; + + public DynamicTopicRegister(TopicExtractor topicExtractor) { + this.topicExtractor = checkNotNull(topicExtractor); + } + + @Override + public List topics(IN in) { + TopicPartition partition = topicExtractor.extract(in, cachedMetadataProvider); + String topicName = partition.getFullTopicName(); + + if (partition.isPartition()) { + return singletonList(topicName); + } else { + try { + List topics = partitionsCache.getIfPresent(topicName); + if (topics == null) { + topics = queryTopics(topicName); + partitionsCache.put(topicName, topics); + } + + return topics; + } catch (PulsarAdminException e) { + throw new FlinkRuntimeException( + "Failed to query Pulsar topic partitions.", e.getCause()); + } + } + } + + private List queryTopics(String topic) throws PulsarAdminException { + TopicMetadata metadata = notExistedMetadataProvider.query(topic); + if (metadata.isPartitioned()) { + int partitionSize = metadata.getPartitionSize(); + List partitions = new ArrayList<>(partitionSize); + for (int i = 0; i < partitionSize; i++) { + partitions.add(topicNameWithPartition(topic, i)); + } + return partitions; + } else { + return singletonList(topic); + } + } + + @Override + public void open(SinkConfiguration sinkConfiguration, ProcessingTimeService timeService) { + // Initialize Pulsar admin instance. + this.pulsarAdmin = createAdmin(sinkConfiguration); + this.cachedMetadataProvider = + new CachedTopicMetadataProvider(pulsarAdmin, sinkConfiguration); + this.notExistedMetadataProvider = + new NotExistedTopicMetadataProvider(pulsarAdmin, sinkConfiguration); + + long refreshInterval = sinkConfiguration.getTopicMetadataRefreshInterval(); + if (refreshInterval <= 0) { + refreshInterval = Long.MAX_VALUE; + } + this.partitionsCache = + CacheBuilder.newBuilder() + .expireAfterWrite(refreshInterval, TimeUnit.MILLISECONDS) + .maximumSize(1000) + .build(); + + // Open the topic extractor instance. + topicExtractor.open(sinkConfiguration); + } + + @Override + public void close() throws IOException { + if (pulsarAdmin != null) { + pulsarAdmin.close(); + } + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/EmptyTopicRegister.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/EmptyTopicRegister.java new file mode 100644 index 0000000000000..b350b3cac3471 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/EmptyTopicRegister.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.sink.writer.topic.register; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.operators.ProcessingTimeService; +import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; +import org.apache.flink.connector.pulsar.sink.writer.topic.TopicRegister; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +/** The topic register which would do nothing for just return an empty topic partitions. */ +@Internal +public class EmptyTopicRegister implements TopicRegister { + private static final long serialVersionUID = -9199261243659491097L; + + @Override + public List topics(IN in) { + return Collections.emptyList(); + } + + @Override + public void open(SinkConfiguration sinkConfiguration, ProcessingTimeService timeService) { + // Nothing to do. + } + + @Override + public void close() throws IOException { + // Nothing to do. + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicMetadataListener.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/FixedTopicRegister.java similarity index 71% rename from flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicMetadataListener.java rename to flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/FixedTopicRegister.java index acd1c61f3b12d..fb5d3658060aa 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicMetadataListener.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/FixedTopicRegister.java @@ -16,45 +16,46 @@ * limitations under the License. */ -package org.apache.flink.connector.pulsar.sink.writer.topic; +package org.apache.flink.connector.pulsar.sink.writer.topic.register; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.operators.ProcessingTimeService; import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; +import org.apache.flink.connector.pulsar.sink.writer.topic.TopicRegister; +import org.apache.flink.connector.pulsar.sink.writer.topic.metadata.NotExistedTopicMetadataProvider; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata; import org.apache.flink.shaded.guava30.com.google.common.base.Objects; import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; -import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.Closeable; import java.io.IOException; -import java.io.Serializable; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -import static java.util.Collections.emptyList; import static org.apache.flink.connector.pulsar.common.config.PulsarClientFactory.createAdmin; import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyAdmin; import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.isPartitioned; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicName; import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; +import static org.apache.pulsar.common.partition.PartitionedTopicMetadata.NON_PARTITIONED; /** - * We need the latest topic metadata for making sure the newly created topic partitions would be - * used by the Pulsar sink. This routing policy would be different compared with Pulsar Client - * built-in logic. We use Flink's ProcessingTimer as the executor. + * We need the latest topic metadata for making sure the Pulsar sink would use the newly created + * topic partitions. This routing policy would be different compared with Pulsar Client built-in + * logic. We use Flink's ProcessingTimer as the executor. */ @Internal -public class TopicMetadataListener implements Serializable, Closeable { +public class FixedTopicRegister implements TopicRegister { private static final long serialVersionUID = 6186948471557507522L; - private static final Logger LOG = LoggerFactory.getLogger(TopicMetadataListener.class); + private static final Logger LOG = LoggerFactory.getLogger(FixedTopicRegister.class); private final ImmutableList partitionedTopics; private final Map topicMetadata; @@ -64,12 +65,9 @@ public class TopicMetadataListener implements Serializable, Closeable { private transient PulsarAdmin pulsarAdmin; private transient Long topicMetadataRefreshInterval; private transient ProcessingTimeService timeService; + private transient NotExistedTopicMetadataProvider metadataProvider; - public TopicMetadataListener() { - this(emptyList()); - } - - public TopicMetadataListener(List topics) { + public FixedTopicRegister(List topics) { List partitions = new ArrayList<>(topics.size()); Map metadata = new HashMap<>(topics.size()); for (String topic : topics) { @@ -86,7 +84,7 @@ public TopicMetadataListener(List topics) { this.availableTopics = ImmutableList.of(); } - /** Register the topic metadata update in process time service. */ + @Override public void open(SinkConfiguration sinkConfiguration, ProcessingTimeService timeService) { if (topicMetadata.isEmpty()) { LOG.info("No topics have been provided, skip listener initialize."); @@ -97,26 +95,31 @@ public void open(SinkConfiguration sinkConfiguration, ProcessingTimeService time this.pulsarAdmin = createAdmin(sinkConfiguration); this.topicMetadataRefreshInterval = sinkConfiguration.getTopicMetadataRefreshInterval(); this.timeService = timeService; + this.metadataProvider = new NotExistedTopicMetadataProvider(pulsarAdmin, sinkConfiguration); // Initialize the topic metadata. Quit if fail to connect to Pulsar. sneakyAdmin(this::updateTopicMetadata); - // Register time service. - triggerNextTopicMetadataUpdate(true); + // Register time service, if user enable the topic metadata update. + if (topicMetadataRefreshInterval > 0) { + triggerNextTopicMetadataUpdate(true); + } } - /** - * Return all the available topic partitions. We would recalculate the partitions if the topic - * metadata has been changed. Otherwise, we would return the cached result for better - * performance. - */ - public List availableTopics() { + @Override + public List topics(IN in) { if (availableTopics.isEmpty() && (!partitionedTopics.isEmpty() || !topicMetadata.isEmpty())) { List results = new ArrayList<>(); for (Map.Entry entry : topicMetadata.entrySet()) { - for (int i = 0; i < entry.getValue(); i++) { - results.add(topicNameWithPartition(entry.getKey(), i)); + int partitionNums = entry.getValue(); + // Get all topics from partitioned and non-partitioned topic names + if (partitionNums == NON_PARTITIONED) { + results.add(topicName(entry.getKey())); + } else { + for (int i = 0; i < partitionNums; i++) { + results.add(topicNameWithPartition(entry.getKey(), i)); + } } } @@ -155,12 +158,11 @@ private void updateTopicMetadata() throws PulsarAdminException { for (Map.Entry entry : topicMetadata.entrySet()) { String topic = entry.getKey(); - PartitionedTopicMetadata metadata = - pulsarAdmin.topics().getPartitionedTopicMetadata(topic); + TopicMetadata metadata = metadataProvider.query(topic); // Update topic metadata if it has been changed. - if (!Objects.equal(entry.getValue(), metadata.partitions)) { - entry.setValue(metadata.partitions); + if (!Objects.equal(entry.getValue(), metadata.getPartitionSize())) { + entry.setValue(metadata.getPartitionSize()); shouldUpdate = true; } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java index a6c48d14bc87e..48c30ff4e7d16 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java @@ -32,7 +32,8 @@ import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumStateSerializer; import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumerator; -import org.apache.flink.connector.pulsar.source.enumerator.SplitsAssignmentState; +import org.apache.flink.connector.pulsar.source.enumerator.assigner.SplitAssigner; +import org.apache.flink.connector.pulsar.source.enumerator.assigner.SplitAssignerFactory; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; import org.apache.flink.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; @@ -44,6 +45,10 @@ import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplitSerializer; import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.pulsar.client.api.CryptoKeyReader; + +import javax.annotation.Nullable; + /** * The Source implementation of Pulsar. Please use a {@link PulsarSourceBuilder} to construct a * {@link PulsarSource}. The following example shows how to create a PulsarSource emitting records @@ -90,18 +95,21 @@ public final class PulsarSource /** The pulsar deserialization schema used for deserializing message. */ private final PulsarDeserializationSchema deserializationSchema; + @Nullable private final CryptoKeyReader cryptoKeyReader; + /** * The constructor for PulsarSource, it's package protected for forcing using {@link * PulsarSourceBuilder}. */ - PulsarSource( + public PulsarSource( SourceConfiguration sourceConfiguration, PulsarSubscriber subscriber, RangeGenerator rangeGenerator, StartCursor startCursor, StopCursor stopCursor, Boundedness boundedness, - PulsarDeserializationSchema deserializationSchema) { + PulsarDeserializationSchema deserializationSchema, + @Nullable CryptoKeyReader cryptoKeyReader) { this.sourceConfiguration = sourceConfiguration; this.subscriber = subscriber; this.rangeGenerator = rangeGenerator; @@ -109,6 +117,7 @@ public final class PulsarSource this.stopCursor = stopCursor; this.boundedness = boundedness; this.deserializationSchema = deserializationSchema; + this.cryptoKeyReader = cryptoKeyReader; } /** @@ -135,22 +144,21 @@ public SourceReader createReader(SourceReaderContext deserializationSchema.open(initializationContext, sourceConfiguration); return PulsarSourceReaderFactory.create( - readerContext, deserializationSchema, sourceConfiguration); + readerContext, deserializationSchema, sourceConfiguration, cryptoKeyReader); } @Internal @Override public SplitEnumerator createEnumerator( SplitEnumeratorContext enumContext) { - SplitsAssignmentState assignmentState = - new SplitsAssignmentState(stopCursor, sourceConfiguration); + SplitAssigner splitAssigner = SplitAssignerFactory.create(stopCursor, sourceConfiguration); return new PulsarSourceEnumerator( subscriber, startCursor, rangeGenerator, sourceConfiguration, enumContext, - assignmentState); + splitAssigner); } @Internal @@ -158,15 +166,15 @@ public SplitEnumerator createEnumer public SplitEnumerator restoreEnumerator( SplitEnumeratorContext enumContext, PulsarSourceEnumState checkpoint) { - SplitsAssignmentState assignmentState = - new SplitsAssignmentState(stopCursor, sourceConfiguration, checkpoint); + SplitAssigner splitAssigner = + SplitAssignerFactory.create(stopCursor, sourceConfiguration, checkpoint); return new PulsarSourceEnumerator( subscriber, startCursor, rangeGenerator, sourceConfiguration, enumContext, - assignmentState); + splitAssigner); } @Internal diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java index 517286d8341a3..5c6bb624cdb1a 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java @@ -25,6 +25,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.pulsar.common.config.PulsarConfigBuilder; import org.apache.flink.connector.pulsar.common.config.PulsarOptions; +import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSchemaWrapper; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; @@ -36,12 +37,15 @@ import org.apache.flink.connector.pulsar.source.enumerator.topic.range.UniformRangeGenerator; import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; +import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.RegexSubscriptionMode; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.util.Arrays; import java.util.List; import java.util.Properties; @@ -54,6 +58,7 @@ import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_CONSUMER_NAME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_READ_SCHEMA_EVOLUTION; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_READ_TRANSACTION_TIMEOUT; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_TYPE; @@ -97,7 +102,7 @@ *

To stop the connector user has to disable the auto partition discovery. As auto partition * discovery always expected new splits to come and not exiting. To disable auto partition * discovery, use builder.setConfig({@link - * PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS}, -1). + * PulsarSourceOptions#PULSAR_PARTITION_DISCOVERY_INTERVAL_MS}, -1). * *

{@code
  * PulsarSource source = PulsarSource
@@ -125,6 +130,7 @@ public final class PulsarSourceBuilder {
     private StopCursor stopCursor;
     private Boundedness boundedness;
     private PulsarDeserializationSchema deserializationSchema;
+    @Nullable private CryptoKeyReader cryptoKeyReader;
 
     // private builder constructor.
     PulsarSourceBuilder() {
@@ -266,13 +272,22 @@ public PulsarSourceBuilder setTopicPattern(
     }
 
     /**
-     * The consumer name is informative and it can be used to identify a particular consumer
+     * The consumer name is informative, and it can be used to identify a particular consumer
      * instance from the topic stats.
      */
     public PulsarSourceBuilder setConsumerName(String consumerName) {
         return setConfig(PULSAR_CONSUMER_NAME, consumerName);
     }
 
+    /**
+     * If you enable this option, we would consume and deserialize the message by using Pulsar
+     * {@link Schema}.
+     */
+    public PulsarSourceBuilder enableSchemaEvolution() {
+        configBuilder.set(PULSAR_READ_SCHEMA_EVOLUTION, true);
+        return this;
+    }
+
     /**
      * Set a topic range generator for Key_Shared subscription.
      *
@@ -321,7 +336,7 @@ public PulsarSourceBuilder setStartCursor(StartCursor startCursor) {
      * 

To stop the connector user has to disable the auto partition discovery. As auto partition * discovery always expected new splits to come and not exiting. To disable auto partition * discovery, use builder.setConfig({@link - * PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS}, -1). + * PulsarSourceOptions#PULSAR_PARTITION_DISCOVERY_INTERVAL_MS}, -1). * * @param stopCursor The {@link StopCursor} to specify the stopping offset. * @return this PulsarSourceBuilder. @@ -334,7 +349,7 @@ public PulsarSourceBuilder setUnboundedStopCursor(StopCursor stopCursor) { } /** - * By default the PulsarSource is set to run in {@link Boundedness#CONTINUOUS_UNBOUNDED} manner + * By default, the PulsarSource is set to run in {@link Boundedness#CONTINUOUS_UNBOUNDED} manner * and thus never stops until the Flink job fails or is canceled. To let the PulsarSource run in * {@link Boundedness#BOUNDED} manner and stops at some point, one can set an {@link StopCursor} * to specify the stopping offsets for each partition. When all the partitions have reached @@ -369,6 +384,18 @@ public PulsarSourceBuilder setDeserializationSchema( return self; } + /** + * Sets a {@link CryptoKeyReader}. Configure the key reader to be used to decrypt the message + * payloads. + * + * @param cryptoKeyReader CryptoKeyReader object + * @return this PulsarSourceBuilder. + */ + public PulsarSourceBuilder setCryptoKeyReader(CryptoKeyReader cryptoKeyReader) { + this.cryptoKeyReader = checkNotNull(cryptoKeyReader); + return this; + } + /** * Set an arbitrary property for the PulsarSource and Pulsar Consumer. The valid keys can be * found in {@link PulsarSourceOptions} and {@link PulsarOptions}. @@ -470,12 +497,29 @@ public PulsarSource build() { } } + // Schema evolution validation. + if (Boolean.TRUE.equals(configBuilder.get(PULSAR_READ_SCHEMA_EVOLUTION))) { + checkState( + deserializationSchema instanceof PulsarSchemaWrapper, + "When enabling schema evolution, you must provide a Pulsar Schema in PulsarDeserializationSchema."); + } else if (deserializationSchema instanceof PulsarSchemaWrapper) { + LOG.info( + "It seems like you want to read message using Pulsar Schema." + + " You can enableSchemaEvolution for using this feature." + + " We would use Schema.BYTES as the default schema if you don't enable this option."); + } + if (!configBuilder.contains(PULSAR_CONSUMER_NAME)) { LOG.warn( "We recommend set a readable consumer name through setConsumerName(String) in production mode."); + } else { + String consumerName = configBuilder.get(PULSAR_CONSUMER_NAME); + if (!consumerName.contains("%s")) { + configBuilder.override(PULSAR_CONSUMER_NAME, consumerName + " - %s"); + } } - // Since these implementation could be a lambda, make sure they are serializable. + // Since these implementations could be a lambda, make sure they are serializable. checkState(isSerializable(startCursor), "StartCursor isn't serializable"); checkState(isSerializable(stopCursor), "StopCursor isn't serializable"); checkState(isSerializable(rangeGenerator), "RangeGenerator isn't serializable"); @@ -491,7 +535,8 @@ public PulsarSource build() { startCursor, stopCursor, boundedness, - deserializationSchema); + deserializationSchema, + cryptoKeyReader); } // ------------- private helpers -------------- diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java index 39a73974f5c0a..6e66abbc01b5e 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java @@ -26,6 +26,7 @@ import org.apache.flink.configuration.description.Description; import org.apache.flink.connector.pulsar.common.config.PulsarOptions; import org.apache.flink.connector.pulsar.source.config.CursorVerification; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.pulsar.client.api.ConsumerCryptoFailureAction; import org.apache.pulsar.client.api.SubscriptionInitialPosition; @@ -147,6 +148,14 @@ private PulsarSourceOptions() { public static final ConfigOption PULSAR_TRANSACTION_TIMEOUT_MILLIS = PULSAR_READ_TRANSACTION_TIMEOUT; + public static final ConfigOption PULSAR_DEFAULT_FETCH_TIME = + ConfigOptions.key(SOURCE_CONFIG_PREFIX + "defaultFetchTime") + .longType() + .defaultValue(100L) + .withDescription( + "The time (in ms) for fetching messages from Pulsar. If time exceed and no message returned from Pulsar." + + " We would consider there is no record at the current topic and stop fetch until next switch."); + public static final ConfigOption PULSAR_MAX_FETCH_TIME = ConfigOptions.key(SOURCE_CONFIG_PREFIX + "maxFetchTime") .longType() @@ -188,6 +197,17 @@ private PulsarSourceOptions() { " A possible solution is to adjust the retention settings in Pulsar or ignoring the check result.") .build()); + public static final ConfigOption PULSAR_READ_SCHEMA_EVOLUTION = + ConfigOptions.key(SOURCE_CONFIG_PREFIX + "enableSchemaEvolution") + .booleanType() + .defaultValue(false) + .withDescription( + Description.builder() + .text( + "If you enable this option, we would consume and deserialize the message by using Pulsar's %s.", + code("Schema")) + .build()); + /////////////////////////////////////////////////////////////////////////////// // // The configuration for ConsumerConfigurationData part. @@ -503,6 +523,12 @@ private PulsarSourceOptions() { code("PulsarClientException")) .build()); + /** + * @deprecated This option would be reset by {@link StartCursor}, no need to use it anymore. + * Pulsar didn't support this config option before 1.10.1, so we have to remove this config + * option. + */ + @Deprecated public static final ConfigOption PULSAR_SUBSCRIPTION_INITIAL_POSITION = ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "subscriptionInitialPosition") diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/PulsarSourceConfigUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/PulsarSourceConfigUtils.java index adb8a03df0328..0a4dc31e8d320 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/PulsarSourceConfigUtils.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/PulsarSourceConfigUtils.java @@ -29,6 +29,7 @@ import java.util.Map; import java.util.Optional; +import java.util.UUID; import static java.util.concurrent.TimeUnit.MICROSECONDS; import static java.util.concurrent.TimeUnit.MILLISECONDS; @@ -58,7 +59,6 @@ import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_REPLICATE_SUBSCRIPTION_STATE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_RETRY_ENABLE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_RETRY_LETTER_TOPIC; -import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_INITIAL_POSITION; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_MODE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_TYPE; @@ -106,11 +106,12 @@ public static ConsumerBuilder createConsumerBuilder( configuration.useOption( PULSAR_MAX_TOTAL_RECEIVER_QUEUE_SIZE_ACROSS_PARTITIONS, builder::maxTotalReceiverQueueSizeAcrossPartitions); - configuration.useOption(PULSAR_CONSUMER_NAME, builder::consumerName); + configuration.useOption( + PULSAR_CONSUMER_NAME, + consumerName -> String.format(consumerName, UUID.randomUUID()), + builder::consumerName); configuration.useOption(PULSAR_READ_COMPACTED, builder::readCompacted); configuration.useOption(PULSAR_PRIORITY_LEVEL, builder::priorityLevel); - configuration.useOption( - PULSAR_SUBSCRIPTION_INITIAL_POSITION, builder::subscriptionInitialPosition); createDeadLetterPolicy(configuration).ifPresent(builder::deadLetterPolicy); configuration.useOption( PULSAR_AUTO_UPDATE_PARTITIONS_INTERVAL_SECONDS, diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java index 806fe4a418b23..b3291531b2852 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java @@ -23,10 +23,12 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.flink.connector.pulsar.common.config.PulsarConfiguration; +import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSchemaWrapper; import org.apache.flink.connector.pulsar.source.enumerator.cursor.CursorPosition; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.pulsar.client.api.ConsumerBuilder; +import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionMode; import org.apache.pulsar.client.api.SubscriptionType; @@ -35,17 +37,19 @@ import static org.apache.flink.connector.base.source.reader.SourceReaderOptions.ELEMENT_QUEUE_CAPACITY; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_AUTO_COMMIT_CURSOR_INTERVAL; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_DEFAULT_FETCH_TIME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_RECORDS; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_TIME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_READ_SCHEMA_EVOLUTION; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_READ_TRANSACTION_TIMEOUT; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_MODE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_TYPE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_VERIFY_INITIAL_OFFSETS; -/** The configure class for pulsar source. */ +/** The configuration class for pulsar source. */ @PublicEvolving public class SourceConfiguration extends PulsarConfiguration { private static final long serialVersionUID = 8488507275800787580L; @@ -53,8 +57,10 @@ public class SourceConfiguration extends PulsarConfiguration { private final int messageQueueCapacity; private final long partitionDiscoveryIntervalMs; private final boolean enableAutoAcknowledgeMessage; + private final boolean enableSchemaEvolution; private final long autoCommitCursorInterval; private final long transactionTimeoutMillis; + private final Duration defaultFetchTime; private final Duration maxFetchTime; private final int maxFetchRecords; private final CursorVerification verifyInitialOffsets; @@ -68,8 +74,10 @@ public SourceConfiguration(Configuration configuration) { this.messageQueueCapacity = getInteger(ELEMENT_QUEUE_CAPACITY); this.partitionDiscoveryIntervalMs = get(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS); this.enableAutoAcknowledgeMessage = get(PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE); + this.enableSchemaEvolution = get(PULSAR_READ_SCHEMA_EVOLUTION); this.autoCommitCursorInterval = get(PULSAR_AUTO_COMMIT_CURSOR_INTERVAL); this.transactionTimeoutMillis = get(PULSAR_READ_TRANSACTION_TIMEOUT); + this.defaultFetchTime = get(PULSAR_DEFAULT_FETCH_TIME, Duration::ofMillis); this.maxFetchTime = get(PULSAR_MAX_FETCH_TIME, Duration::ofMillis); this.maxFetchRecords = get(PULSAR_MAX_FETCH_RECORDS); this.verifyInitialOffsets = get(PULSAR_VERIFY_INITIAL_OFFSETS); @@ -83,6 +91,10 @@ public int getMessageQueueCapacity() { return messageQueueCapacity; } + /** + * We would override the interval into a negative number when we set the connector with bounded + * stop cursor. + */ public boolean isEnablePartitionDiscovery() { return getPartitionDiscoveryIntervalMs() > 0; } @@ -107,6 +119,14 @@ public boolean isEnableAutoAcknowledgeMessage() { return enableAutoAcknowledgeMessage; } + /** + * If we should deserialize the message with a specified Pulsar {@link Schema} instead the + * default {@link Schema#BYTES}. This switch is only used for {@link PulsarSchemaWrapper}. + */ + public boolean isEnableSchemaEvolution() { + return enableSchemaEvolution; + } + /** * The interval in millis for acknowledge message when you enable {@link * #isEnableAutoAcknowledgeMessage} and use {@link SubscriptionType#Failover} or {@link @@ -127,6 +147,14 @@ public long getTransactionTimeoutMillis() { return transactionTimeoutMillis; } + /** + * The fetch time for polling one message. We would stop polling message and return the message + * in {@link RecordsWithSplitIds} when timeout and no message consumed. + */ + public Duration getDefaultFetchTime() { + return defaultFetchTime; + } + /** * The fetch time for flink split reader polling message. We would stop polling message and * return the message in {@link RecordsWithSplitIds} when timeout or exceed the {@link @@ -179,16 +207,6 @@ public SubscriptionMode getSubscriptionMode() { return subscriptionMode; } - /** Convert the subscription into a readable str. */ - public String getSubscriptionDesc() { - return getSubscriptionName() - + "(" - + getSubscriptionType() - + "," - + getSubscriptionMode() - + ")"; - } - @Override public boolean equals(Object o) { if (this == o) { @@ -205,8 +223,9 @@ public boolean equals(Object o) { && enableAutoAcknowledgeMessage == that.enableAutoAcknowledgeMessage && autoCommitCursorInterval == that.autoCommitCursorInterval && transactionTimeoutMillis == that.transactionTimeoutMillis - && maxFetchRecords == that.maxFetchRecords + && Objects.equals(defaultFetchTime, that.defaultFetchTime) && Objects.equals(maxFetchTime, that.maxFetchTime) + && maxFetchRecords == that.maxFetchRecords && verifyInitialOffsets == that.verifyInitialOffsets && Objects.equals(subscriptionName, that.subscriptionName) && subscriptionType == that.subscriptionType @@ -221,6 +240,7 @@ public int hashCode() { enableAutoAcknowledgeMessage, autoCommitCursorInterval, transactionTimeoutMillis, + defaultFetchTime, maxFetchTime, maxFetchRecords, verifyInitialOffsets, diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumState.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumState.java index dbab9e2178123..56bbbd20a32a2 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumState.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumState.java @@ -18,15 +18,18 @@ package org.apache.flink.connector.pulsar.source.enumerator; +import org.apache.flink.connector.pulsar.source.enumerator.assigner.SplitAssigner; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; +import java.util.HashMap; +import java.util.HashSet; import java.util.Map; import java.util.Set; /** * The state class for pulsar source enumerator, used for storing the split state. This class is - * managed and controlled by {@link SplitsAssignmentState}. + * managed and controlled by {@link SplitAssigner}. */ public class PulsarSourceEnumState { @@ -46,11 +49,12 @@ public class PulsarSourceEnumState { private final Map> sharedPendingPartitionSplits; /** - * A {@link PulsarPartitionSplit} should be assigned for all flink readers. Using this map for - * recording assign status. + * It is used for Shared subscription. Every {@link PulsarPartitionSplit} should be assigned for + * all flink readers. Using this map for recording assign status. */ private final Map> readerAssignedSplits; + /** The pipeline has been triggered and topic partitions have been assigned to readers. */ private final boolean initialized; public PulsarSourceEnumState( @@ -85,4 +89,10 @@ public Map> getReaderAssignedSplits() { public boolean isInitialized() { return initialized; } + + /** The initial assignment state for Pulsar. */ + public static PulsarSourceEnumState initialState() { + return new PulsarSourceEnumState( + new HashSet<>(), new HashSet<>(), new HashMap<>(), new HashMap<>(), false); + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumerator.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumerator.java index 7890dcf1847f7..84ffbcd1a7224 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumerator.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumerator.java @@ -22,40 +22,29 @@ import org.apache.flink.api.connector.source.SplitEnumerator; import org.apache.flink.api.connector.source.SplitEnumeratorContext; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.assigner.SplitAssigner; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.CursorPosition; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.flink.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange; import org.apache.flink.connector.pulsar.source.enumerator.topic.range.RangeGenerator; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.flink.util.FlinkRuntimeException; import org.apache.pulsar.client.admin.PulsarAdmin; -import org.apache.pulsar.client.api.Consumer; -import org.apache.pulsar.client.api.ConsumerBuilder; -import org.apache.pulsar.client.api.KeySharedPolicy; -import org.apache.pulsar.client.api.KeySharedPolicy.KeySharedPolicySticky; -import org.apache.pulsar.client.api.PulsarClient; -import org.apache.pulsar.client.api.PulsarClientException; -import org.apache.pulsar.client.api.Range; -import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.api.MessageId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nullable; import java.util.ArrayList; -import java.util.HashSet; import java.util.List; import java.util.Set; import static java.util.Collections.singletonList; import static org.apache.flink.connector.pulsar.common.config.PulsarClientFactory.createAdmin; -import static org.apache.flink.connector.pulsar.common.config.PulsarClientFactory.createClient; -import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; -import static org.apache.flink.connector.pulsar.source.config.CursorVerification.FAIL_ON_MISMATCH; -import static org.apache.flink.connector.pulsar.source.config.PulsarSourceConfigUtils.createConsumerBuilder; +import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyAdmin; /** The enumerator class for pulsar source. */ @Internal @@ -65,13 +54,12 @@ public class PulsarSourceEnumerator private static final Logger LOG = LoggerFactory.getLogger(PulsarSourceEnumerator.class); private final PulsarAdmin pulsarAdmin; - private final PulsarClient pulsarClient; private final PulsarSubscriber subscriber; private final StartCursor startCursor; private final RangeGenerator rangeGenerator; private final SourceConfiguration sourceConfiguration; private final SplitEnumeratorContext context; - private final SplitsAssignmentState assignmentState; + private final SplitAssigner splitAssigner; public PulsarSourceEnumerator( PulsarSubscriber subscriber, @@ -79,15 +67,14 @@ public PulsarSourceEnumerator( RangeGenerator rangeGenerator, SourceConfiguration sourceConfiguration, SplitEnumeratorContext context, - SplitsAssignmentState assignmentState) { + SplitAssigner splitAssigner) { this.pulsarAdmin = createAdmin(sourceConfiguration); - this.pulsarClient = createClient(sourceConfiguration); this.subscriber = subscriber; this.startCursor = startCursor; this.rangeGenerator = rangeGenerator; this.sourceConfiguration = sourceConfiguration; this.context = context; - this.assignmentState = assignmentState; + this.splitAssigner = splitAssigner; } @Override @@ -99,7 +86,7 @@ public void start() { LOG.info( "Starting the PulsarSourceEnumerator for subscription {} " + "with partition discovery interval of {} ms.", - sourceConfiguration.getSubscriptionDesc(), + subscriptionDesc(), sourceConfiguration.getPartitionDiscoveryIntervalMs()); context.callAsync( this::getSubscribedTopicPartitions, @@ -110,7 +97,7 @@ public void start() { LOG.info( "Starting the PulsarSourceEnumerator for subscription {} " + "without periodic partition discovery.", - sourceConfiguration.getSubscriptionDesc()); + subscriptionDesc()); context.callAsync(this::getSubscribedTopicPartitions, this::checkPartitionChanges); } } @@ -123,9 +110,9 @@ public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname @Override public void addSplitsBack(List splits, int subtaskId) { // Put the split back to current pending splits. - assignmentState.putSplitsBackToPendingList(splits, subtaskId); + splitAssigner.addSplitsBack(splits, subtaskId); - // If the failed subtask has already restarted, we need to assign pending splits to it + // If the failed subtask has already restarted, we need to assign pending splits to it. if (context.registeredReaders().containsKey(subtaskId)) { assignPendingPartitionSplits(singletonList(subtaskId)); } @@ -136,13 +123,13 @@ public void addReader(int subtaskId) { LOG.debug( "Adding reader {} to PulsarSourceEnumerator for subscription {}.", subtaskId, - sourceConfiguration.getSubscriptionDesc()); + subscriptionDesc()); assignPendingPartitionSplits(singletonList(subtaskId)); } @Override public PulsarSourceEnumState snapshotState(long checkpointId) { - return assignmentState.snapshotState(); + return splitAssigner.snapshotState(); } @Override @@ -154,6 +141,15 @@ public void close() { // ----------------- private methods ------------------- + /** Convert the subscription into a readable string. */ + private String subscriptionDesc() { + return String.format( + "%s(%s,%s)", + sourceConfiguration.getSubscriptionName(), + sourceConfiguration.getSubscriptionType(), + sourceConfiguration.getSubscriptionMode()); + } + /** * List subscribed topic partitions on Pulsar cluster. * @@ -164,54 +160,7 @@ public void close() { */ private Set getSubscribedTopicPartitions() { int parallelism = context.currentParallelism(); - Set partitions = - subscriber.getSubscribedTopicPartitions(pulsarAdmin, rangeGenerator, parallelism); - - // Seek start position for given partitions. - seekStartPosition(partitions); - - return partitions; - } - - private void seekStartPosition(Set partitions) { - ConsumerBuilder consumerBuilder = consumerBuilder(); - Set seekedTopics = new HashSet<>(); - - for (TopicPartition partition : partitions) { - String topicName = partition.getFullTopicName(); - if (!assignmentState.containsTopic(topicName) && seekedTopics.add(topicName)) { - try (Consumer consumer = - sneakyClient(() -> consumerBuilder.clone().topic(topicName).subscribe())) { - startCursor.seekPosition( - partition.getTopic(), partition.getPartitionId(), consumer); - } catch (PulsarClientException e) { - if (sourceConfiguration.getVerifyInitialOffsets() == FAIL_ON_MISMATCH) { - throw new IllegalArgumentException(e); - } else { - // WARN_ON_MISMATCH would just print this warning message. - // No need to print the stacktrace. - LOG.warn( - "Failed to set initial consuming position for partition {}", - partition, - e); - } - } - } - } - } - - private ConsumerBuilder consumerBuilder() { - ConsumerBuilder builder = - createConsumerBuilder(pulsarClient, Schema.BYTES, sourceConfiguration); - if (sourceConfiguration.getSubscriptionType() == SubscriptionType.Key_Shared) { - Range range = TopicRange.createFullRange().toPulsarRange(); - KeySharedPolicySticky keySharedPolicy = KeySharedPolicy.stickyHashRange().ranges(range); - // Force this consume use sticky hash range in Key_Shared subscription. - // Pulsar won't remove old message dispatcher before 2.8.2 release. - builder.keySharedPolicy(keySharedPolicy); - } - - return builder; + return subscriber.getSubscribedTopicPartitions(pulsarAdmin, rangeGenerator, parallelism); } /** @@ -230,13 +179,55 @@ private void checkPartitionChanges(Set fetchedPartitions, Throwa } // Append the partitions into current assignment state. - assignmentState.appendTopicPartitions(fetchedPartitions); - List registeredReaders = new ArrayList<>(context.registeredReaders().keySet()); + List newPartitions = + splitAssigner.registerTopicPartitions(fetchedPartitions); + createSubscription(newPartitions); // Assign the new readers. + List registeredReaders = new ArrayList<>(context.registeredReaders().keySet()); assignPendingPartitionSplits(registeredReaders); } + /** Create subscription on topic partition if it doesn't exist. */ + private void createSubscription(List newPartitions) { + for (TopicPartition partition : newPartitions) { + String topicName = partition.getFullTopicName(); + String subscriptionName = sourceConfiguration.getSubscriptionName(); + + List subscriptions = + sneakyAdmin(() -> pulsarAdmin.topics().getSubscriptions(topicName)); + if (!subscriptions.contains(subscriptionName)) { + CursorPosition position = + startCursor.position(partition.getTopic(), partition.getPartitionId()); + MessageId initialPosition = queryInitialPosition(topicName, position); + + sneakyAdmin( + () -> + pulsarAdmin + .topics() + .createSubscription( + topicName, subscriptionName, initialPosition)); + } + } + } + + /** Query the available message id from Pulsar. */ + private MessageId queryInitialPosition(String topicName, CursorPosition position) { + CursorPosition.Type type = position.getType(); + if (type == CursorPosition.Type.TIMESTAMP) { + return sneakyAdmin( + () -> + pulsarAdmin + .topics() + .getMessageIdByTimestamp(topicName, position.getTimestamp())); + } else if (type == CursorPosition.Type.MESSAGE_ID) { + return position.getMessageId(); + } else { + throw new UnsupportedOperationException("We don't support this seek type " + type); + } + } + + /** Query the unassigned splits and assign them to the available readers. */ private void assignPendingPartitionSplits(List pendingReaders) { // Validate the reader. pendingReaders.forEach( @@ -248,17 +239,19 @@ private void assignPendingPartitionSplits(List pendingReaders) { }); // Assign splits to downstream readers. - assignmentState.assignSplits(pendingReaders).ifPresent(context::assignSplits); + splitAssigner.createAssignment(pendingReaders).ifPresent(context::assignSplits); // If periodically partition discovery is disabled and the initializing discovery has done, - // signal NoMoreSplitsEvent to pending readers - if (assignmentState.noMoreNewPartitionSplits()) { - LOG.debug( - "No more PulsarPartitionSplits to assign." - + " Sending NoMoreSplitsEvent to reader {} in subscription {}.", - pendingReaders, - sourceConfiguration.getSubscriptionDesc()); - pendingReaders.forEach(this.context::signalNoMoreSplits); + // signal NoMoreSplitsEvent to pending readers. + for (Integer reader : pendingReaders) { + if (splitAssigner.noMoreSplits(reader)) { + LOG.debug( + "No more PulsarPartitionSplits to assign." + + " Sending NoMoreSplitsEvent to reader {} in subscription {}.", + reader, + subscriptionDesc()); + context.signalNoMoreSplits(reader); + } } } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/SplitsAssignmentState.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/SplitsAssignmentState.java deleted file mode 100644 index cbc4826583a53..0000000000000 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/SplitsAssignmentState.java +++ /dev/null @@ -1,239 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.connector.pulsar.source.enumerator; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.flink.api.connector.source.SplitsAssignment; -import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; -import org.apache.flink.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; -import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; -import org.apache.flink.util.InstantiationUtil; - -import org.apache.pulsar.client.api.SubscriptionType; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.Set; - -/** The state class for recording the split assignment. */ -@Internal -public class SplitsAssignmentState { - - private final StopCursor stopCursor; - private final SourceConfiguration sourceConfiguration; - - // The dynamic states for checkpoint. - private final Set appendedPartitions; - // This pending splits is used for Key_Shared, Failover, Exclusive subscription. - private final Set pendingPartitionSplits; - // These two fields are used for Shared subscription. - private final Map> sharedPendingPartitionSplits; - private final Map> readerAssignedSplits; - private boolean initialized; - - public SplitsAssignmentState(StopCursor stopCursor, SourceConfiguration sourceConfiguration) { - this.stopCursor = stopCursor; - this.sourceConfiguration = sourceConfiguration; - this.appendedPartitions = new HashSet<>(); - this.pendingPartitionSplits = new HashSet<>(); - this.sharedPendingPartitionSplits = new HashMap<>(); - this.readerAssignedSplits = new HashMap<>(); - this.initialized = false; - } - - public SplitsAssignmentState( - StopCursor stopCursor, - SourceConfiguration sourceConfiguration, - PulsarSourceEnumState sourceEnumState) { - this.stopCursor = stopCursor; - this.sourceConfiguration = sourceConfiguration; - this.appendedPartitions = sourceEnumState.getAppendedPartitions(); - this.pendingPartitionSplits = sourceEnumState.getPendingPartitionSplits(); - this.sharedPendingPartitionSplits = sourceEnumState.getSharedPendingPartitionSplits(); - this.readerAssignedSplits = sourceEnumState.getReaderAssignedSplits(); - this.initialized = sourceEnumState.isInitialized(); - } - - public PulsarSourceEnumState snapshotState() { - return new PulsarSourceEnumState( - appendedPartitions, - pendingPartitionSplits, - sharedPendingPartitionSplits, - readerAssignedSplits, - initialized); - } - - /** - * Append the new fetched partitions to current state. We would generate pending source split - * for downstream pulsar readers. Since the {@link SplitEnumeratorContext} don't support put the - * split back to enumerator, we don't support partition deletion. - * - * @param fetchedPartitions The partitions from the {@link PulsarSubscriber}. - */ - public void appendTopicPartitions(Set fetchedPartitions) { - for (TopicPartition partition : fetchedPartitions) { - // If this partition is a new partition. - if (!appendedPartitions.contains(partition)) { - if (!sharePartition()) { - // Create a split and add it to pending list. - pendingPartitionSplits.add(createSplit(partition)); - } - - // Shared subscription don't create splits, we just register partitions. - appendedPartitions.add(partition); - } - } - - // Update this initialize flag. - if (!initialized) { - this.initialized = true; - } - } - - public boolean containsTopic(String topicName) { - return appendedPartitions.stream() - .anyMatch(partition -> Objects.equals(partition.getFullTopicName(), topicName)); - } - - /** Put these splits back to pending list. */ - public void putSplitsBackToPendingList(List splits, int readerId) { - if (!sharePartition()) { - // Put these splits back to normal pending list. - pendingPartitionSplits.addAll(splits); - } else { - // Put the splits back to shared pending list. - Set pending = - sharedPendingPartitionSplits.computeIfAbsent(readerId, id -> new HashSet<>()); - pending.addAll(splits); - } - } - - public Optional> assignSplits( - List pendingReaders) { - // Avoid empty readers assign. - if (pendingReaders.isEmpty()) { - return Optional.empty(); - } - - Map> assignMap; - - // We extract the assign logic into two method for better readability. - if (!sharePartition()) { - assignMap = assignNormalSplits(pendingReaders); - } else { - assignMap = assignSharedSplits(pendingReaders); - } - - if (assignMap.isEmpty()) { - return Optional.empty(); - } else { - return Optional.of(new SplitsAssignment<>(assignMap)); - } - } - - /** - * @return It would return true only if periodically partition discovery is disabled, the - * initializing partition discovery has finished AND there is no pending splits for - * assignment. - */ - public boolean noMoreNewPartitionSplits() { - return !sourceConfiguration.isEnablePartitionDiscovery() - && initialized - && pendingPartitionSplits.isEmpty(); - } - - // ----------------- private methods ------------------- - - /** The splits don't shared for all the readers. */ - private Map> assignNormalSplits( - List pendingReaders) { - Map> assignMap = new HashMap<>(); - - // Drain a list of splits. - List pendingSplits = drainPendingPartitionsSplits(); - for (int i = 0; i < pendingSplits.size(); i++) { - PulsarPartitionSplit split = pendingSplits.get(i); - int readerId = pendingReaders.get(i % pendingReaders.size()); - assignMap.computeIfAbsent(readerId, id -> new ArrayList<>()).add(split); - } - - return assignMap; - } - - /** Every split would be shared among available readers. */ - private Map> assignSharedSplits( - List pendingReaders) { - Map> assignMap = new HashMap<>(); - - // Drain the splits from share pending list. - for (Integer reader : pendingReaders) { - Set pendingSplits = sharedPendingPartitionSplits.remove(reader); - if (pendingSplits == null) { - pendingSplits = new HashSet<>(); - } - - Set assignedSplits = - readerAssignedSplits.computeIfAbsent(reader, r -> new HashSet<>()); - - for (TopicPartition partition : appendedPartitions) { - String partitionName = partition.toString(); - if (!assignedSplits.contains(partitionName)) { - pendingSplits.add(createSplit(partition)); - assignedSplits.add(partitionName); - } - } - - if (!pendingSplits.isEmpty()) { - assignMap.put(reader, new ArrayList<>(pendingSplits)); - } - } - - return assignMap; - } - - private PulsarPartitionSplit createSplit(TopicPartition partition) { - try { - StopCursor stop = InstantiationUtil.clone(stopCursor); - return new PulsarPartitionSplit(partition, stop); - } catch (IOException | ClassNotFoundException e) { - throw new IllegalStateException(e); - } - } - - private List drainPendingPartitionsSplits() { - List splits = new ArrayList<>(pendingPartitionSplits); - pendingPartitionSplits.clear(); - - return splits; - } - - /** {@link SubscriptionType#Shared} mode should share a same split for all the readers. */ - private boolean sharePartition() { - return sourceConfiguration.getSubscriptionType() == SubscriptionType.Shared; - } -} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssigner.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssigner.java new file mode 100644 index 0000000000000..087e96157d655 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssigner.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.source.enumerator.assigner; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; +import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; + +import org.apache.pulsar.client.api.SubscriptionType; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +/** + * This assigner is used for {@link SubscriptionType#Failover}, {@link SubscriptionType#Exclusive} + * and {@link SubscriptionType#Key_Shared} subscriptions. + */ +@Internal +public class NonSharedSplitAssigner implements SplitAssigner { + private static final long serialVersionUID = 8412586087991597092L; + + private final StopCursor stopCursor; + private final boolean enablePartitionDiscovery; + + // These fields would be saved into checkpoint. + + private final Set appendedPartitions; + private final Set pendingPartitionSplits; + private boolean initialized; + + public NonSharedSplitAssigner( + StopCursor stopCursor, + SourceConfiguration sourceConfiguration, + PulsarSourceEnumState sourceEnumState) { + this.stopCursor = stopCursor; + this.enablePartitionDiscovery = sourceConfiguration.isEnablePartitionDiscovery(); + this.appendedPartitions = sourceEnumState.getAppendedPartitions(); + this.pendingPartitionSplits = sourceEnumState.getPendingPartitionSplits(); + this.initialized = sourceEnumState.isInitialized(); + } + + @Override + public List registerTopicPartitions(Set fetchedPartitions) { + List newPartitions = new ArrayList<>(); + + for (TopicPartition partition : fetchedPartitions) { + if (!appendedPartitions.contains(partition)) { + pendingPartitionSplits.add(new PulsarPartitionSplit(partition, stopCursor)); + appendedPartitions.add(partition); + newPartitions.add(partition); + } + } + + if (!initialized) { + initialized = true; + } + + return newPartitions; + } + + @Override + public void addSplitsBack(List splits, int subtaskId) { + pendingPartitionSplits.addAll(splits); + } + + @Override + public Optional> createAssignment( + List readers) { + if (pendingPartitionSplits.isEmpty() || readers.isEmpty()) { + return Optional.empty(); + } + + Map> assignMap = new HashMap<>(); + + List partitionSplits = new ArrayList<>(pendingPartitionSplits); + int readerCount = readers.size(); + for (int i = 0; i < partitionSplits.size(); i++) { + int index = i % readerCount; + Integer readerId = readers.get(index); + PulsarPartitionSplit split = partitionSplits.get(i); + assignMap.computeIfAbsent(readerId, id -> new ArrayList<>()).add(split); + } + pendingPartitionSplits.clear(); + + return Optional.of(new SplitsAssignment<>(assignMap)); + } + + @Override + public boolean noMoreSplits(Integer reader) { + return !enablePartitionDiscovery && initialized && pendingPartitionSplits.isEmpty(); + } + + @Override + public PulsarSourceEnumState snapshotState() { + return new PulsarSourceEnumState( + appendedPartitions, + pendingPartitionSplits, + new HashMap<>(), + new HashMap<>(), + initialized); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssigner.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssigner.java new file mode 100644 index 0000000000000..48d75c8dee30d --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssigner.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.source.enumerator.assigner; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; +import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; + +import org.apache.pulsar.client.api.SubscriptionType; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +/** This assigner is used for {@link SubscriptionType#Shared} subscriptions. */ +@Internal +public class SharedSplitAssigner implements SplitAssigner { + private static final long serialVersionUID = 8468503133499402491L; + + private final StopCursor stopCursor; + private final boolean enablePartitionDiscovery; + + // These fields would be saved into checkpoint. + + private final Set appendedPartitions; + private final Map> sharedPendingPartitionSplits; + private final Map> readerAssignedSplits; + private boolean initialized; + + public SharedSplitAssigner( + StopCursor stopCursor, + SourceConfiguration sourceConfiguration, + PulsarSourceEnumState sourceEnumState) { + this.stopCursor = stopCursor; + this.enablePartitionDiscovery = sourceConfiguration.isEnablePartitionDiscovery(); + this.appendedPartitions = sourceEnumState.getAppendedPartitions(); + this.sharedPendingPartitionSplits = sourceEnumState.getSharedPendingPartitionSplits(); + this.readerAssignedSplits = sourceEnumState.getReaderAssignedSplits(); + this.initialized = sourceEnumState.isInitialized(); + } + + @Override + public List registerTopicPartitions(Set fetchedPartitions) { + List newPartitions = new ArrayList<>(); + + for (TopicPartition partition : fetchedPartitions) { + if (!appendedPartitions.contains(partition)) { + appendedPartitions.add(partition); + newPartitions.add(partition); + } + } + + if (!initialized) { + initialized = true; + } + + return newPartitions; + } + + @Override + public void addSplitsBack(List splits, int subtaskId) { + Set pendingPartitionSplits = + sharedPendingPartitionSplits.computeIfAbsent(subtaskId, id -> new HashSet<>()); + pendingPartitionSplits.addAll(splits); + } + + @Override + public Optional> createAssignment( + List readers) { + if (readers.isEmpty()) { + return Optional.empty(); + } + + Map> assignMap = new HashMap<>(); + for (Integer reader : readers) { + Set pendingSplits = sharedPendingPartitionSplits.remove(reader); + if (pendingSplits == null) { + pendingSplits = new HashSet<>(); + } + + Set assignedSplits = + readerAssignedSplits.computeIfAbsent(reader, r -> new HashSet<>()); + + for (TopicPartition partition : appendedPartitions) { + String partitionName = partition.toString(); + if (!assignedSplits.contains(partitionName)) { + pendingSplits.add(new PulsarPartitionSplit(partition, stopCursor)); + assignedSplits.add(partitionName); + } + } + + if (!pendingSplits.isEmpty()) { + assignMap.put(reader, new ArrayList<>(pendingSplits)); + } + } + + if (assignMap.isEmpty()) { + return Optional.empty(); + } else { + return Optional.of(new SplitsAssignment<>(assignMap)); + } + } + + @Override + public boolean noMoreSplits(Integer reader) { + Set pendingSplits = sharedPendingPartitionSplits.get(reader); + Set assignedSplits = readerAssignedSplits.get(reader); + + return !enablePartitionDiscovery + && initialized + && (pendingSplits == null || pendingSplits.isEmpty()) + && (assignedSplits != null && assignedSplits.size() == appendedPartitions.size()); + } + + @Override + public PulsarSourceEnumState snapshotState() { + return new PulsarSourceEnumState( + appendedPartitions, + new HashSet<>(), + sharedPendingPartitionSplits, + readerAssignedSplits, + initialized); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssigner.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssigner.java new file mode 100644 index 0000000000000..bc03f5103fd88 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssigner.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.source.enumerator.assigner; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; +import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; + +import java.io.Serializable; +import java.util.List; +import java.util.Optional; +import java.util.Set; + +/** + * The split assigner for different subscription. We would spread all the splits to different + * readers and store all the state into checkpoint. + */ +@Internal +public interface SplitAssigner extends Serializable { + + /** + * Add the current available partitions into assigner. + * + * @param fetchedPartitions The available partitions queried from Pulsar broker. + * @return New topic partitions compare to previous registered partitions. + */ + List registerTopicPartitions(Set fetchedPartitions); + + /** + * Add a split back to the split assigner if the reader fails. We would try to reassign the + * split or add it to the pending list. + */ + void addSplitsBack(List splits, int subtaskId); + + /** Create a split assignment from the current readers. */ + Optional> createAssignment(List readers); + + /** + * It would return true only if periodically partition discovery is disabled, the initializing + * partition discovery has finished AND there is no pending splits for assignment. + */ + boolean noMoreSplits(Integer reader); + + /** Snapshot the current assign state into checkpoint. */ + PulsarSourceEnumState snapshotState(); +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerFactory.java new file mode 100644 index 0000000000000..3e6ebccb49b4c --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerFactory.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.source.enumerator.assigner; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; + +import org.apache.pulsar.client.api.SubscriptionType; + +import static org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState.initialState; +import static org.apache.pulsar.client.api.SubscriptionType.Exclusive; +import static org.apache.pulsar.client.api.SubscriptionType.Failover; +import static org.apache.pulsar.client.api.SubscriptionType.Key_Shared; +import static org.apache.pulsar.client.api.SubscriptionType.Shared; + +/** The factory for creating split assigner. */ +@Internal +public final class SplitAssignerFactory { + + private SplitAssignerFactory() { + // No public constructor. + } + + /** Create blank assigner. */ + public static SplitAssigner create( + StopCursor stopCursor, SourceConfiguration sourceConfiguration) { + return create(stopCursor, sourceConfiguration, initialState()); + } + + /** Create assigner from checkpoint state. */ + public static SplitAssigner create( + StopCursor stopCursor, + SourceConfiguration sourceConfiguration, + PulsarSourceEnumState sourceEnumState) { + SubscriptionType subscriptionType = sourceConfiguration.getSubscriptionType(); + if (subscriptionType == Exclusive + || subscriptionType == Failover + || subscriptionType == Key_Shared) { + return new NonSharedSplitAssigner(stopCursor, sourceConfiguration, sourceEnumState); + } else if (subscriptionType == Shared) { + return new SharedSplitAssigner(stopCursor, sourceConfiguration, sourceEnumState); + } else { + throw new IllegalArgumentException( + "We don't support this subscription type: " + subscriptionType); + } + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/CursorPosition.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/CursorPosition.java index a2aaff629066d..c965ff962f807 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/CursorPosition.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/CursorPosition.java @@ -18,18 +18,16 @@ package org.apache.flink.connector.pulsar.source.enumerator.cursor; +import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.PulsarClientException; - -import javax.annotation.Nullable; import java.io.Serializable; -/** The class for defining the start or stop position. */ +/** + * The class for defining the start or stop position. We only expose the constructor for end user. + */ @PublicEvolving public final class CursorPosition implements Serializable { private static final long serialVersionUID = -802405183307684549L; @@ -40,34 +38,31 @@ public final class CursorPosition implements Serializable { private final Long timestamp; - public CursorPosition(@Nullable MessageId messageId) { + public CursorPosition(MessageId messageId) { this.type = Type.MESSAGE_ID; this.messageId = messageId; this.timestamp = null; } - public CursorPosition(@Nullable Long timestamp) { + public CursorPosition(Long timestamp) { this.type = Type.TIMESTAMP; this.messageId = null; this.timestamp = timestamp; } - @VisibleForTesting + @Internal + public Type getType() { + return type; + } + + @Internal public MessageId getMessageId() { return messageId; } - /** Pulsar consumer could be subscribed by the position. */ - public void seekPosition(Consumer consumer) throws PulsarClientException { - if (type == Type.MESSAGE_ID) { - consumer.seek(messageId); - } else { - if (timestamp != null) { - consumer.seek(timestamp); - } else { - consumer.seek(System.currentTimeMillis()); - } - } + @Internal + public Long getTimestamp() { + return timestamp; } @Override @@ -82,6 +77,7 @@ public String toString() { /** * The position type for reader to choose whether timestamp or message id as the start position. */ + @Internal public enum Type { TIMESTAMP, diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/MessageIdUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/MessageIdUtils.java new file mode 100644 index 0000000000000..a8c3a6b2ef284 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/MessageIdUtils.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.source.enumerator.cursor; + +import org.apache.flink.annotation.Internal; + +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.impl.BatchMessageIdImpl; +import org.apache.pulsar.client.impl.MessageIdImpl; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** The helper class for Pulsar's message id. */ +@Internal +public final class MessageIdUtils { + + private MessageIdUtils() { + // No public constructor. + } + + /** + * The implementation from this + * code snippet to get next message id. + */ + public static MessageId nextMessageId(MessageId messageId) { + MessageIdImpl idImpl = unwrapMessageId(messageId); + + if (idImpl.getEntryId() < 0) { + return newMessageId(idImpl.getLedgerId(), 0, idImpl.getPartitionIndex()); + } else { + return newMessageId( + idImpl.getLedgerId(), idImpl.getEntryId() + 1, idImpl.getPartitionIndex()); + } + } + + /** + * Convert the message id interface to its backend implementation. And check if it's a batch + * message id. We don't support the batch message for its low performance now. + */ + public static MessageIdImpl unwrapMessageId(MessageId messageId) { + MessageIdImpl idImpl = MessageIdImpl.convertToMessageIdImpl(messageId); + if (idImpl instanceof BatchMessageIdImpl) { + int batchSize = ((BatchMessageIdImpl) idImpl).getBatchSize(); + checkArgument(batchSize == 1, "We only support normal message id currently."); + } + + return idImpl; + } + + /** Hide the message id implementation. */ + public static MessageId newMessageId(long ledgerId, long entryId, int partitionIndex) { + return new MessageIdImpl(ledgerId, entryId, partitionIndex); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StartCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StartCursor.java index af35319a5a71e..9c1d699a269f7 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StartCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StartCursor.java @@ -22,9 +22,7 @@ import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.MessageIdStartCursor; import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.TimestampStartCursor; -import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SubscriptionType; import java.io.Serializable; @@ -43,13 +41,6 @@ public interface StartCursor extends Serializable { CursorPosition position(String topic, int partitionId); - /** Helper method for seek the right position for given pulsar consumer. */ - default void seekPosition(String topic, int partitionId, Consumer consumer) - throws PulsarClientException { - CursorPosition position = position(topic, partitionId); - position.seekPosition(consumer); - } - // --------------------------- Static Factory Methods ----------------------------- static StartCursor defaultStartCursor() { @@ -64,19 +55,38 @@ static StartCursor latest() { return fromMessageId(MessageId.latest); } + /** + * Find the available message id and start consuming from it. The given message is included in + * the consuming result by default if you provide a specified message id instead of {@link + * MessageId#earliest} or {@link MessageId#latest}. + */ static StartCursor fromMessageId(MessageId messageId) { return fromMessageId(messageId, true); } /** * @param messageId Find the available message id and start consuming from it. - * @param inclusive {@code true} would include the given message id. + * @param inclusive {@code true} would include the given message id if it's not the {@link + * MessageId#earliest} or {@link MessageId#latest}. */ static StartCursor fromMessageId(MessageId messageId, boolean inclusive) { return new MessageIdStartCursor(messageId, inclusive); } + /** + * This method is designed for seeking message from event time. But Pulsar didn't support + * seeking from message time, instead, it would seek the position from publish time. We only + * keep this method for backward compatible. + * + * @deprecated Use {@link #fromPublishTime(long)} instead. + */ + @Deprecated static StartCursor fromMessageTime(long timestamp) { - return new TimestampStartCursor(timestamp); + return new TimestampStartCursor(timestamp, true); + } + + /** Seek the start position by using message publish time. */ + static StartCursor fromPublishTime(long timestamp) { + return new TimestampStartCursor(timestamp, true); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursor.java index aaec14352dcf2..d44c78fcf1a44 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursor.java @@ -19,10 +19,11 @@ package org.apache.flink.connector.pulsar.source.enumerator.cursor; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.stop.EventTimestampStopCursor; import org.apache.flink.connector.pulsar.source.enumerator.cursor.stop.LatestMessageStopCursor; import org.apache.flink.connector.pulsar.source.enumerator.cursor.stop.MessageIdStopCursor; import org.apache.flink.connector.pulsar.source.enumerator.cursor.stop.NeverStopCursor; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.stop.TimestampStopCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.stop.PublishTimestampStopCursor; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; import org.apache.pulsar.client.admin.PulsarAdmin; @@ -43,11 +44,55 @@ public interface StopCursor extends Serializable { /** The open method for the cursor initializer. This method could be executed multiple times. */ default void open(PulsarAdmin admin, TopicPartition partition) {} - /** - * Determine whether to pause consumption on the current message by the returned boolean value. - * The message presented in method argument wouldn't be consumed if the return result is true. - */ - boolean shouldStop(Message message); + /** Determine whether to pause consumption on the current message by the returned enum. */ + StopCondition shouldStop(Message message); + + /** The conditional for control the stop behavior of the pulsar source. */ + @PublicEvolving + enum StopCondition { + + /** This message should be included in the result. */ + CONTINUE, + /** This message should be included in the result and stop consuming. */ + EXACTLY, + /** Stop consuming, the given message wouldn't be included in the result. */ + TERMINATE; + + /** + * Common methods for comparing the message id. + * + * @param desired The stop goal of the message id. + * @param current The upcoming message id. + * @param inclusive Should the desired message be included in the consuming result. + */ + public static StopCondition compare( + MessageId desired, MessageId current, boolean inclusive) { + if (current.compareTo(desired) < 0) { + return StopCondition.CONTINUE; + } else if (current.compareTo(desired) == 0) { + return inclusive ? StopCondition.EXACTLY : StopCondition.TERMINATE; + } else { + return StopCondition.TERMINATE; + } + } + + /** + * Common methods for comparing the message time. + * + * @param desired The stop goal of the message time. + * @param current The upcoming message time. + * @param inclusive Should the desired message be included in the consuming result. + */ + public static StopCondition compare(long desired, long current, boolean inclusive) { + if (current < desired) { + return StopCondition.CONTINUE; + } else if (current == desired) { + return inclusive ? StopCondition.EXACTLY : StopCondition.TERMINATE; + } else { + return StopCondition.TERMINATE; + } + } + } // --------------------------- Static Factory Methods ----------------------------- @@ -60,18 +105,52 @@ static StopCursor never() { } static StopCursor latest() { - return new LatestMessageStopCursor(); + return new LatestMessageStopCursor(true); } + /** + * Stop consuming when the messageId is equal or greater than the specified messageId. Message + * that is equal to the specified messageId will not be consumed. + */ static StopCursor atMessageId(MessageId messageId) { - return new MessageIdStopCursor(messageId); + if (MessageId.latest.equals(messageId)) { + return new LatestMessageStopCursor(false); + } else { + return new MessageIdStopCursor(messageId, false); + } } + /** + * Stop consuming when the messageId is greater than the specified messageId. Message that is + * equal to the specified messageId will be consumed. + */ static StopCursor afterMessageId(MessageId messageId) { - return new MessageIdStopCursor(messageId, false); + if (MessageId.latest.equals(messageId)) { + return new LatestMessageStopCursor(true); + } else { + return new MessageIdStopCursor(messageId, true); + } } + /** Stop consuming when message eventTime is greater than or equals the specified timestamp. */ static StopCursor atEventTime(long timestamp) { - return new TimestampStopCursor(timestamp); + return new EventTimestampStopCursor(timestamp, false); + } + + /** Stop consuming when message eventTime is greater than the specified timestamp. */ + static StopCursor afterEventTime(long timestamp) { + return new EventTimestampStopCursor(timestamp, true); + } + + /** + * Stop consuming when message publishTime is greater than or equals the specified timestamp. + */ + static StopCursor atPublishTime(long timestamp) { + return new PublishTimestampStopCursor(timestamp, false); + } + + /** Stop consuming when message publishTime is greater than the specified timestamp. */ + static StopCursor afterPublishTime(long timestamp) { + return new PublishTimestampStopCursor(timestamp, true); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java index f8079608cc65a..1f26e834749fa 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java @@ -25,9 +25,12 @@ import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.impl.MessageIdImpl; -import static org.apache.flink.util.Preconditions.checkState; +import java.util.Objects; -/** This cursor would left pulsar start consuming from a specific message id. */ +import static org.apache.flink.connector.pulsar.source.enumerator.cursor.MessageIdUtils.nextMessageId; +import static org.apache.flink.connector.pulsar.source.enumerator.cursor.MessageIdUtils.unwrapMessageId; + +/** This cursor would leave pulsar start consuming from a specific message id. */ public class MessageIdStartCursor implements StartCursor { private static final long serialVersionUID = -8057345435887170111L; @@ -43,23 +46,16 @@ public class MessageIdStartCursor implements StartCursor { * code for understanding pulsar internal logic. * * @param messageId The message id for start position. - * @param inclusive Should we include the start message id in consuming result. + * @param inclusive Whether we include the start message id in consuming result. This works only + * if we provide a specified message id instead of {@link MessageId#earliest} or {@link + * MessageId#latest}. */ public MessageIdStartCursor(MessageId messageId, boolean inclusive) { - if (inclusive) { - this.messageId = messageId; + MessageIdImpl idImpl = unwrapMessageId(messageId); + if (MessageId.earliest.equals(idImpl) || MessageId.latest.equals(idImpl) || inclusive) { + this.messageId = idImpl; } else { - checkState( - messageId instanceof MessageIdImpl, - "We only support normal message id and batch message id."); - MessageIdImpl id = (MessageIdImpl) messageId; - if (MessageId.earliest.equals(messageId) || MessageId.latest.equals(messageId)) { - this.messageId = messageId; - } else { - this.messageId = - new MessageIdImpl( - id.getLedgerId(), id.getEntryId() + 1, id.getPartitionIndex()); - } + this.messageId = nextMessageId(idImpl); } } @@ -67,4 +63,21 @@ public MessageIdStartCursor(MessageId messageId, boolean inclusive) { public CursorPosition position(String topic, int partitionId) { return new CursorPosition(messageId); } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MessageIdStartCursor that = (MessageIdStartCursor) o; + return Objects.equals(messageId, that.messageId); + } + + @Override + public int hashCode() { + return Objects.hash(messageId); + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java index 88f3b5a0b4c74..065de66b39fed 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java @@ -21,18 +21,37 @@ import org.apache.flink.connector.pulsar.source.enumerator.cursor.CursorPosition; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; -/** This cursor would left pulsar start consuming from a specific timestamp. */ +import java.util.Objects; + +/** This cursor would left pulsar start consuming from a specific publish timestamp. */ public class TimestampStartCursor implements StartCursor { private static final long serialVersionUID = 5170578885838095320L; private final long timestamp; - public TimestampStartCursor(long timestamp) { - this.timestamp = timestamp; + public TimestampStartCursor(long timestamp, boolean inclusive) { + this.timestamp = inclusive ? timestamp : timestamp + 1; } @Override public CursorPosition position(String topic, int partitionId) { return new CursorPosition(timestamp); } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TimestampStartCursor that = (TimestampStartCursor) o; + return timestamp == that.timestamp; + } + + @Override + public int hashCode() { + return Objects.hash(timestamp); + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java new file mode 100644 index 0000000000000..dfaadbed0a4b3 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.source.enumerator.cursor.stop; + +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; + +import org.apache.pulsar.client.api.Message; + +import java.util.Objects; + +/** Stop consuming message at the given event time. */ +public class EventTimestampStopCursor implements StopCursor { + private static final long serialVersionUID = 2391576769339369027L; + + private final long timestamp; + private final boolean inclusive; + + public EventTimestampStopCursor(long timestamp, boolean inclusive) { + this.timestamp = timestamp; + this.inclusive = inclusive; + } + + @Override + public StopCondition shouldStop(Message message) { + long eventTime = message.getEventTime(); + return StopCondition.compare(timestamp, eventTime, inclusive); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + EventTimestampStopCursor that = (EventTimestampStopCursor) o; + return timestamp == that.timestamp && inclusive == that.inclusive; + } + + @Override + public int hashCode() { + return Objects.hash(timestamp, inclusive); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java index 257081f5c8938..0311e4a8fb362 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java @@ -25,29 +25,53 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; +import java.util.Objects; + import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyAdmin; /** * A stop cursor that initialize the position to the latest message id. The offsets initialization * are taken care of by the {@code PulsarPartitionSplitReaderBase} instead of by the {@code - * PulsarSourceEnumerator}. + * PulsarSourceEnumerator}. We would include the latest message available in Pulsar by default. */ public class LatestMessageStopCursor implements StopCursor { private static final long serialVersionUID = 1702059838323965723L; private MessageId messageId; + private final boolean inclusive; + + public LatestMessageStopCursor(boolean inclusive) { + this.inclusive = inclusive; + } + + @Override + public StopCondition shouldStop(Message message) { + MessageId current = message.getMessageId(); + return StopCondition.compare(messageId, current, inclusive); + } @Override public void open(PulsarAdmin admin, TopicPartition partition) { if (messageId == null) { String topic = partition.getFullTopicName(); - messageId = sneakyAdmin(() -> admin.topics().getLastMessageId(topic)); + this.messageId = sneakyAdmin(() -> admin.topics().getLastMessageId(topic)); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; } + LatestMessageStopCursor that = (LatestMessageStopCursor) o; + return inclusive == that.inclusive && Objects.equals(messageId, that.messageId); } @Override - public boolean shouldStop(Message message) { - MessageId id = message.getMessageId(); - return id.compareTo(messageId) >= 0; + public int hashCode() { + return Objects.hash(messageId, inclusive); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java index 7af55a00cc09f..f6ff913199fd1 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java @@ -22,6 +22,14 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.impl.MessageIdImpl; + +import java.util.Objects; + +import static org.apache.flink.connector.pulsar.source.enumerator.cursor.MessageIdUtils.unwrapMessageId; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.pulsar.client.api.MessageId.earliest; +import static org.apache.pulsar.client.api.MessageId.latest; /** * Stop consuming message at a given message id. We use the {@link MessageId#compareTo(Object)} for @@ -32,24 +40,39 @@ public class MessageIdStopCursor implements StopCursor { private final MessageId messageId; - private final boolean exclusive; + private final boolean inclusive; - public MessageIdStopCursor(MessageId messageId) { - this(messageId, true); + public MessageIdStopCursor(MessageId messageId, boolean inclusive) { + MessageIdImpl idImpl = unwrapMessageId(messageId); + checkArgument(!earliest.equals(idImpl), "MessageId.earliest is not supported."); + checkArgument( + !latest.equals(idImpl), + "MessageId.latest is not supported, use LatestMessageStopCursor instead."); + + this.messageId = idImpl; + this.inclusive = inclusive; } - public MessageIdStopCursor(MessageId messageId, boolean exclusive) { - this.messageId = messageId; - this.exclusive = exclusive; + @Override + public StopCondition shouldStop(Message message) { + MessageId current = message.getMessageId(); + return StopCondition.compare(messageId, current, inclusive); } @Override - public boolean shouldStop(Message message) { - MessageId id = message.getMessageId(); - if (exclusive) { - return id.compareTo(messageId) > 0; - } else { - return id.compareTo(messageId) >= 0; + public boolean equals(Object o) { + if (this == o) { + return true; } + if (o == null || getClass() != o.getClass()) { + return false; + } + MessageIdStopCursor that = (MessageIdStopCursor) o; + return inclusive == that.inclusive && Objects.equals(messageId, that.messageId); + } + + @Override + public int hashCode() { + return Objects.hash(messageId, inclusive); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java index ff2c619afb8b0..99a42976d5b92 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java @@ -27,7 +27,16 @@ public class NeverStopCursor implements StopCursor { private static final long serialVersionUID = -3113601090292771786L; @Override - public boolean shouldStop(Message message) { - return false; + public StopCondition shouldStop(Message message) { + return StopCondition.CONTINUE; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + return o != null && getClass() == o.getClass(); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java new file mode 100644 index 0000000000000..c374f90c6a7af --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.source.enumerator.cursor.stop; + +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; + +import org.apache.pulsar.client.api.Message; + +import java.util.Objects; + +/** Stop consuming message at the given publish time. */ +public class PublishTimestampStopCursor implements StopCursor { + private static final long serialVersionUID = 4386276745339324527L; + + private final long timestamp; + private final boolean inclusive; + + public PublishTimestampStopCursor(long timestamp, boolean inclusive) { + this.timestamp = timestamp; + this.inclusive = inclusive; + } + + @Override + public StopCondition shouldStop(Message message) { + long publishTime = message.getPublishTime(); + return StopCondition.compare(timestamp, publishTime, inclusive); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PublishTimestampStopCursor that = (PublishTimestampStopCursor) o; + return timestamp == that.timestamp && inclusive == that.inclusive; + } + + @Override + public int hashCode() { + return Objects.hash(timestamp, inclusive); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/PulsarSubscriber.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/PulsarSubscriber.java index 08ba1faa44214..b8a55bf8a34b8 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/PulsarSubscriber.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/PulsarSubscriber.java @@ -48,7 +48,7 @@ public interface PulsarSubscriber extends Serializable { /** * Get a set of subscribed {@link TopicPartition}s. The method could throw {@link - * IllegalStateException}, a extra try catch is required. + * IllegalStateException}, an extra try catch is required. * * @param pulsarAdmin The admin interface used to retrieve subscribed topic partitions. * @param rangeGenerator The range for different partitions. diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/BasePulsarSubscriber.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/BasePulsarSubscriber.java index d266c98df4253..f9ff0618215b8 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/BasePulsarSubscriber.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/BasePulsarSubscriber.java @@ -20,7 +20,6 @@ import org.apache.flink.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange; @@ -28,10 +27,12 @@ import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import java.util.ArrayList; import java.util.List; -import java.util.stream.IntStream; import static java.util.stream.Collectors.toList; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicName; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition.NON_PARTITION_ID; /** PulsarSubscriber abstract class to simplify Pulsar admin related operations. */ public abstract class BasePulsarSubscriber implements PulsarSubscriber { @@ -39,7 +40,7 @@ public abstract class BasePulsarSubscriber implements PulsarSubscriber { protected TopicMetadata queryTopicMetadata(PulsarAdmin pulsarAdmin, String topicName) { // Drop the complete topic name for a clean partitioned topic name. - String completeTopicName = TopicNameUtils.topicName(topicName); + String completeTopicName = topicName(topicName); try { PartitionedTopicMetadata metadata = pulsarAdmin.topics().getPartitionedTopicMetadata(completeTopicName); @@ -60,21 +61,18 @@ protected List toTopicPartitions( if (!metadata.isPartitioned()) { // For non-partitioned topic. return ranges.stream() - .map(range -> new TopicPartition(metadata.getName(), -1, range)) + .map(range -> new TopicPartition(metadata.getName(), NON_PARTITION_ID, range)) .collect(toList()); } else { - return IntStream.range(0, metadata.getPartitionSize()) - .boxed() - .flatMap( - partitionId -> - ranges.stream() - .map( - range -> - new TopicPartition( - metadata.getName(), - partitionId, - range))) - .collect(toList()); + List partitions = new ArrayList<>(); + for (int i = 0; i < metadata.getPartitionSize(); i++) { + for (TopicRange range : ranges) { + TopicPartition partition = new TopicPartition(metadata.getName(), i, range); + partitions.add(partition); + } + } + + return partitions; } } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/TopicPatternSubscriber.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/TopicPatternSubscriber.java index 472dbde3e352b..d08022f067de3 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/TopicPatternSubscriber.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/TopicPatternSubscriber.java @@ -18,6 +18,7 @@ package org.apache.flink.connector.pulsar.source.enumerator.subscriber.impl; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange; import org.apache.flink.connector.pulsar.source.enumerator.topic.range.RangeGenerator; @@ -35,6 +36,7 @@ import java.util.regex.Pattern; import static java.util.stream.Collectors.toSet; +import static org.apache.flink.shaded.guava30.com.google.common.base.Predicates.not; /** Subscribe to matching topics based on topic pattern. */ public class TopicPatternSubscriber extends BasePulsarSubscriber { @@ -63,6 +65,7 @@ public Set getSubscribedTopicPartitions( .namespaces() .getTopics(namespace) .parallelStream() + .filter(not(TopicNameUtils::isInternal)) .filter(this::matchesSubscriptionMode) .filter(topic -> topicPattern.matcher(topic).find()) .map(topic -> queryTopicMetadata(pulsarAdmin, topic)) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicNameUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicNameUtils.java index b5d814a56013c..6a35ad6a58e37 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicNameUtils.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicNameUtils.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList; +import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableSet; import org.apache.pulsar.common.naming.TopicName; @@ -30,13 +31,35 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.regex.Pattern; import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.pulsar.common.naming.NamespaceName.SYSTEM_NAMESPACE; +import static org.apache.pulsar.common.naming.TopicDomain.persistent; /** util for topic name. */ @Internal public final class TopicNameUtils { + private static final Pattern HEARTBEAT_NAMESPACE_PATTERN = + Pattern.compile("pulsar/[^/]+/([^:]+:\\d+)"); + private static final Pattern HEARTBEAT_NAMESPACE_PATTERN_V2 = + Pattern.compile("pulsar/([^:]+:\\d+)"); + private static final Pattern SLA_NAMESPACE_PATTERN = + Pattern.compile("sla-monitor" + "/[^/]+/([^:]+:\\d+)"); + private static final Set SYSTEM_TOPIC_NAMES = + ImmutableSet.of( + "__change_events", + "__transaction_buffer_snapshot", + "__pending_ack_state", + "__transaction_pending_ack"); + + private static final String TRANSACTION_COORDINATOR_ASSIGN_PREFIX = + TopicName.get(persistent.value(), SYSTEM_NAMESPACE, "transaction_coordinator_assign") + .toString(); + private static final String TRANSACTION_COORDINATOR_LOG_PREFIX = + TopicName.get(persistent.value(), SYSTEM_NAMESPACE, "__transaction_log_").toString(); + private TopicNameUtils() { // No public constructor. } @@ -87,4 +110,29 @@ public static List distinctTopics(List topics) { return builder.build(); } + + /** + * This method is refactored from {@code BrokerService} in pulsar-broker which is not available + * in Pulsar client. We have to put it here and self maintained. Since these topic names would + * never be changed for backward compatible, we only need to add new topic names after version + * bump. + * + * @see BrokerService#isSystemTopic + */ + public static boolean isInternal(String topic) { + // A topic name instance without partition information. + String topicName = topicName(topic); + TopicName topicInstance = TopicName.get(topicName); + String localName = topicInstance.getLocalName(); + String namespace = topicInstance.getNamespace(); + + return namespace.equals(SYSTEM_NAMESPACE.toString()) + || SLA_NAMESPACE_PATTERN.matcher(namespace).matches() + || HEARTBEAT_NAMESPACE_PATTERN.matcher(namespace).matches() + || HEARTBEAT_NAMESPACE_PATTERN_V2.matcher(namespace).matches() + || SYSTEM_TOPIC_NAMES.contains(localName) + || topicName.startsWith(TRANSACTION_COORDINATOR_ASSIGN_PREFIX) + || topicName.startsWith(TRANSACTION_COORDINATOR_LOG_PREFIX); + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicPartition.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicPartition.java index b3035cde8485e..371e7dfabf707 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicPartition.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicPartition.java @@ -18,6 +18,7 @@ package org.apache.flink.connector.pulsar.source.enumerator.topic; +import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; @@ -29,6 +30,7 @@ import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicName; import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.createFullRange; import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -39,6 +41,12 @@ public class TopicPartition implements Serializable { private static final long serialVersionUID = -1474354741550810953L; + /** + * If {@link TopicPartition#getPartitionId()} is equal to this. This topic partition wouldn't be + * a partition instance. It would be a top topic name. + */ + public static final int NON_PARTITION_ID = -1; + /** * The topic name of the pulsar. It would be a full topic name, if your don't provide the tenant * and namespace, we would add them automatically. @@ -58,6 +66,17 @@ public class TopicPartition implements Serializable { */ private final TopicRange range; + /** Create a top-level topic without partition information. */ + public TopicPartition(String topic) { + this(topic, NON_PARTITION_ID); + } + + /** Create a topic partition without key hash range. */ + public TopicPartition(String topic, int partitionId) { + this(topic, partitionId, createFullRange()); + } + + @Internal public TopicPartition(String topic, int partitionId, TopicRange range) { this.topic = topicName(checkNotNull(topic)); this.partitionId = partitionId; @@ -72,12 +91,17 @@ public int getPartitionId() { return partitionId; } + /** @return Is this a partition instance or a topic instance? */ + public boolean isPartition() { + return partitionId != NON_PARTITION_ID; + } + /** * Pulsar split the topic partition into a bunch of small topics, we would get the real topic * name by using this method. */ public String getFullTopicName() { - if (partitionId >= 0) { + if (isPartition()) { return topicNameWithPartition(topic, partitionId); } else { return topic; diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicRange.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicRange.java index 5b779224b87c7..09d4882ffe9da 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicRange.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicRange.java @@ -48,6 +48,9 @@ public class TopicRange implements Serializable { /** The end position for hash range, it's 65535. */ public static final int MAX_RANGE = RANGE_SIZE - 1; + /** A full topic range instance for avoiding multiple instance creation. */ + private static final TopicRange FULL_RANGE = new TopicRange(MIN_RANGE, MAX_RANGE); + /** The start of the range, default is zero. */ private final int start; @@ -68,9 +71,9 @@ public Range toPulsarRange() { return new Range(start, end); } - /** Create a topic range which contains the fully hash range. */ + /** Create a topic range which contains the full hash range. */ public static TopicRange createFullRange() { - return new TopicRange(MIN_RANGE, MAX_RANGE); + return FULL_RANGE; } public int getStart() { diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/PulsarSourceReaderFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/PulsarSourceReaderFactory.java index 6a5d51522314e..ab384fba061b5 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/PulsarSourceReaderFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/PulsarSourceReaderFactory.java @@ -23,9 +23,12 @@ import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.connector.pulsar.common.schema.BytesSchema; +import org.apache.flink.connector.pulsar.common.schema.PulsarSchema; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; -import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage; +import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarSchemaWrapper; +import org.apache.flink.connector.pulsar.source.reader.emitter.PulsarRecordEmitter; import org.apache.flink.connector.pulsar.source.reader.source.PulsarOrderedSourceReader; import org.apache.flink.connector.pulsar.source.reader.source.PulsarUnorderedSourceReader; import org.apache.flink.connector.pulsar.source.reader.split.PulsarOrderedPartitionSplitReader; @@ -33,11 +36,16 @@ import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.CryptoKeyReader; +import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient; import org.apache.pulsar.client.impl.PulsarClientImpl; +import javax.annotation.Nullable; + import java.util.function.Supplier; import static org.apache.flink.connector.pulsar.common.config.PulsarClientFactory.createAdmin; @@ -63,32 +71,47 @@ private PulsarSourceReaderFactory() { public static SourceReader create( SourceReaderContext readerContext, PulsarDeserializationSchema deserializationSchema, - SourceConfiguration sourceConfiguration) { + SourceConfiguration sourceConfiguration, + @Nullable CryptoKeyReader cryptoKeyReader) { PulsarClient pulsarClient = createClient(sourceConfiguration); PulsarAdmin pulsarAdmin = createAdmin(sourceConfiguration); + // Choose the right schema to use. + Schema schema; + if (sourceConfiguration.isEnableSchemaEvolution()) { + PulsarSchema pulsarSchema = + ((PulsarSchemaWrapper) deserializationSchema).pulsarSchema(); + schema = new BytesSchema(pulsarSchema); + } else { + schema = Schema.BYTES; + } + // Create a message queue with the predefined source option. int queueCapacity = sourceConfiguration.getMessageQueueCapacity(); - FutureCompletingBlockingQueue>> elementsQueue = + FutureCompletingBlockingQueue>> elementsQueue = new FutureCompletingBlockingQueue<>(queueCapacity); + PulsarRecordEmitter recordEmitter = new PulsarRecordEmitter<>(deserializationSchema); + // Create different pulsar source reader by subscription type. SubscriptionType subscriptionType = sourceConfiguration.getSubscriptionType(); if (subscriptionType == SubscriptionType.Failover || subscriptionType == SubscriptionType.Exclusive) { - // Create a ordered split reader supplier. - Supplier> splitReaderSupplier = + // Create an ordered split reader supplier. + Supplier splitReaderSupplier = () -> - new PulsarOrderedPartitionSplitReader<>( + new PulsarOrderedPartitionSplitReader( pulsarClient, pulsarAdmin, sourceConfiguration, - deserializationSchema); + schema, + cryptoKeyReader); return new PulsarOrderedSourceReader<>( elementsQueue, splitReaderSupplier, + recordEmitter, readerContext, sourceConfiguration, pulsarClient, @@ -102,18 +125,20 @@ public static SourceReader create( throw new IllegalStateException("Transaction is required but didn't enabled"); } - Supplier> splitReaderSupplier = + Supplier splitReaderSupplier = () -> - new PulsarUnorderedPartitionSplitReader<>( + new PulsarUnorderedPartitionSplitReader( pulsarClient, pulsarAdmin, sourceConfiguration, - deserializationSchema, + schema, + cryptoKeyReader, coordinatorClient); return new PulsarUnorderedSourceReader<>( elementsQueue, splitReaderSupplier, + recordEmitter, readerContext, sourceConfiguration, pulsarClient, diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaWrapper.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaWrapper.java index e9b2779d9a99c..2b9841656da36 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaWrapper.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaWrapper.java @@ -55,7 +55,6 @@ public void open(InitializationContext context, SourceConfiguration configuratio public void deserialize(Message message, Collector out) throws Exception { byte[] bytes = message.getData(); T instance = deserializationSchema.deserialize(bytes); - out.collect(instance); } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarSchemaWrapper.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarSchemaWrapper.java index 7926d803d4a11..c515008e4f020 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarSchemaWrapper.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarSchemaWrapper.java @@ -35,10 +35,10 @@ * @param The output type of the message. */ @Internal -class PulsarSchemaWrapper implements PulsarDeserializationSchema { +public class PulsarSchemaWrapper implements PulsarDeserializationSchema { private static final long serialVersionUID = -4864701207257059158L; - /** The serializable pulsar schema, it wrap the schema with type class. */ + /** The serializable pulsar schema, it wraps the schema with type class. */ private final PulsarSchema pulsarSchema; public PulsarSchemaWrapper(PulsarSchema pulsarSchema) { @@ -59,4 +59,8 @@ public TypeInformation getProducedType() { SchemaInfo info = pulsarSchema.getSchemaInfo(); return createTypeInformation(info); } + + public PulsarSchema pulsarSchema() { + return pulsarSchema; + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/emitter/PulsarRecordEmitter.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/emitter/PulsarRecordEmitter.java index f6607f0990ac9..c74a284f05ccd 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/emitter/PulsarRecordEmitter.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/emitter/PulsarRecordEmitter.java @@ -20,10 +20,13 @@ import org.apache.flink.api.connector.source.SourceOutput; import org.apache.flink.connector.base.source.reader.RecordEmitter; -import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage; +import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; import org.apache.flink.connector.pulsar.source.reader.source.PulsarOrderedSourceReader; import org.apache.flink.connector.pulsar.source.reader.source.PulsarUnorderedSourceReader; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplitState; +import org.apache.flink.util.Collector; + +import org.apache.pulsar.client.api.Message; /** * The {@link RecordEmitter} implementation for both {@link PulsarOrderedSourceReader} and {@link @@ -31,15 +34,55 @@ * emitter. */ public class PulsarRecordEmitter - implements RecordEmitter, T, PulsarPartitionSplitState> { + implements RecordEmitter, T, PulsarPartitionSplitState> { + + private final PulsarDeserializationSchema deserializationSchema; + private final SourceOutputWrapper sourceOutputWrapper = new SourceOutputWrapper<>(); + + public PulsarRecordEmitter(PulsarDeserializationSchema deserializationSchema) { + this.deserializationSchema = deserializationSchema; + } @Override public void emitRecord( - PulsarMessage element, SourceOutput output, PulsarPartitionSplitState splitState) + Message element, SourceOutput output, PulsarPartitionSplitState splitState) throws Exception { - // Sink the record to source output. - output.collect(element.getValue(), element.getEventTime()); - // Update the split state. - splitState.setLatestConsumedId(element.getId()); + // Update the source output. + sourceOutputWrapper.setSourceOutput(output); + sourceOutputWrapper.setTimestamp(element); + + deserializationSchema.deserialize(element, sourceOutputWrapper); + splitState.setLatestConsumedId(element.getMessageId()); + } + + private static class SourceOutputWrapper implements Collector { + private SourceOutput sourceOutput; + private long timestamp; + + @Override + public void collect(T record) { + if (timestamp > 0) { + sourceOutput.collect(record, timestamp); + } else { + sourceOutput.collect(record); + } + } + + @Override + public void close() { + // Nothing to do here. + } + + private void setSourceOutput(SourceOutput sourceOutput) { + this.sourceOutput = sourceOutput; + } + + /** + * Get the event timestamp from Pulsar. Zero means there is no event time. See {@link + * Message#getEventTime()} to get the reason why it returns zero. + */ + private void setTimestamp(Message message) { + this.timestamp = message.getEventTime(); + } } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarFetcherManagerBase.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarFetcherManagerBase.java index 1f9a35f6c681b..d4ac8a721960b 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarFetcherManagerBase.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarFetcherManagerBase.java @@ -25,9 +25,10 @@ import org.apache.flink.connector.base.source.reader.fetcher.SplitFetcherManager; import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; -import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; +import org.apache.pulsar.client.api.Message; + import java.util.HashMap; import java.util.List; import java.util.Map; @@ -35,14 +36,10 @@ import static java.util.Collections.singletonList; -/** - * Common fetcher manager abstraction for both ordered & unordered message. - * - * @param The decoded message type for flink. - */ +/** Common fetcher manager abstraction for both ordered & unordered message. */ @Internal -public abstract class PulsarFetcherManagerBase - extends SplitFetcherManager, PulsarPartitionSplit> { +public abstract class PulsarFetcherManagerBase + extends SplitFetcherManager, PulsarPartitionSplit> { private final Map splitFetcherMapping = new HashMap<>(); private final Map fetcherStatus = new HashMap<>(); @@ -56,8 +53,8 @@ public abstract class PulsarFetcherManagerBase * @param splitReaderSupplier The factory for the split reader that connects to the source */ protected PulsarFetcherManagerBase( - FutureCompletingBlockingQueue>> elementsQueue, - Supplier, PulsarPartitionSplit>> splitReaderSupplier) { + FutureCompletingBlockingQueue>> elementsQueue, + Supplier, PulsarPartitionSplit>> splitReaderSupplier) { super(elementsQueue, splitReaderSupplier); } @@ -68,7 +65,7 @@ protected PulsarFetcherManagerBase( @Override public void addSplits(List splitsToAdd) { for (PulsarPartitionSplit split : splitsToAdd) { - SplitFetcher, PulsarPartitionSplit> fetcher = + SplitFetcher, PulsarPartitionSplit> fetcher = getOrCreateFetcher(split.splitId()); fetcher.addSplits(singletonList(split)); // This method could be executed multiple times. @@ -76,17 +73,23 @@ public void addSplits(List splitsToAdd) { } } + /** Close the finished split related fetcher. */ + public void closeFetcher(String splitId) { + SplitFetcher, PulsarPartitionSplit> fetcher = getOrCreateFetcher(splitId); + fetcher.shutdown(); + } + @Override - protected void startFetcher(SplitFetcher, PulsarPartitionSplit> fetcher) { + protected void startFetcher(SplitFetcher, PulsarPartitionSplit> fetcher) { if (fetcherStatus.get(fetcher.fetcherId()) != Boolean.TRUE) { fetcherStatus.put(fetcher.fetcherId(), true); super.startFetcher(fetcher); } } - protected SplitFetcher, PulsarPartitionSplit> getOrCreateFetcher( + protected SplitFetcher, PulsarPartitionSplit> getOrCreateFetcher( String splitId) { - SplitFetcher, PulsarPartitionSplit> fetcher; + SplitFetcher, PulsarPartitionSplit> fetcher; Integer fetcherId = splitFetcherMapping.get(splitId); if (fetcherId == null) { diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarOrderedFetcherManager.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarOrderedFetcherManager.java index f8b89ee98bca9..3178be78fb444 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarOrderedFetcherManager.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarOrderedFetcherManager.java @@ -24,11 +24,11 @@ import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; -import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage; import org.apache.flink.connector.pulsar.source.reader.split.PulsarOrderedPartitionSplitReader; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,16 +40,14 @@ * Pulsar's FetcherManager implementation for ordered consuming. This class is needed to help * acknowledge the message to Pulsar using the {@link Consumer} inside the {@link * PulsarOrderedPartitionSplitReader}. - * - * @param The message type for pulsar decoded message. */ @Internal -public class PulsarOrderedFetcherManager extends PulsarFetcherManagerBase { +public class PulsarOrderedFetcherManager extends PulsarFetcherManagerBase { private static final Logger LOG = LoggerFactory.getLogger(PulsarOrderedFetcherManager.class); public PulsarOrderedFetcherManager( - FutureCompletingBlockingQueue>> elementsQueue, - Supplier, PulsarPartitionSplit>> splitReaderSupplier) { + FutureCompletingBlockingQueue>> elementsQueue, + Supplier, PulsarPartitionSplit>> splitReaderSupplier) { super(elementsQueue, splitReaderSupplier); } @@ -57,18 +55,18 @@ public void acknowledgeMessages(Map cursorsToCommit) LOG.debug("Acknowledge messages {}", cursorsToCommit); cursorsToCommit.forEach( (partition, messageId) -> { - SplitFetcher, PulsarPartitionSplit> fetcher = + SplitFetcher, PulsarPartitionSplit> fetcher = getOrCreateFetcher(partition.toString()); triggerAcknowledge(fetcher, partition, messageId); }); } private void triggerAcknowledge( - SplitFetcher, PulsarPartitionSplit> splitFetcher, + SplitFetcher, PulsarPartitionSplit> splitFetcher, TopicPartition partition, MessageId messageId) { - PulsarOrderedPartitionSplitReader splitReader = - (PulsarOrderedPartitionSplitReader) splitFetcher.getSplitReader(); + PulsarOrderedPartitionSplitReader splitReader = + (PulsarOrderedPartitionSplitReader) splitFetcher.getSplitReader(); splitReader.notifyCheckpointComplete(partition, messageId); startFetcher(splitFetcher); } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarUnorderedFetcherManager.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarUnorderedFetcherManager.java index c086a3fba680e..3af00ab68a96d 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarUnorderedFetcherManager.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarUnorderedFetcherManager.java @@ -23,11 +23,11 @@ import org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher; import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; -import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage; import org.apache.flink.connector.pulsar.source.reader.split.PulsarUnorderedPartitionSplitReader; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; import java.util.ArrayList; import java.util.List; @@ -39,15 +39,13 @@ * Pulsar's FetcherManager implementation for unordered consuming. This class is needed to help * acknowledge the message to Pulsar using the {@link Consumer} inside the {@link * PulsarUnorderedPartitionSplitReader}. - * - * @param The message type for pulsar decoded message. */ @Internal -public class PulsarUnorderedFetcherManager extends PulsarFetcherManagerBase { +public class PulsarUnorderedFetcherManager extends PulsarFetcherManagerBase { public PulsarUnorderedFetcherManager( - FutureCompletingBlockingQueue>> elementsQueue, - Supplier, PulsarPartitionSplit>> splitReaderSupplier) { + FutureCompletingBlockingQueue>> elementsQueue, + Supplier, PulsarPartitionSplit>> splitReaderSupplier) { super(elementsQueue, splitReaderSupplier); } @@ -59,8 +57,8 @@ public List snapshotState(long checkpointId) { } private PulsarPartitionSplit snapshotReader( - long checkpointId, SplitReader, PulsarPartitionSplit> splitReader) { - return ((PulsarUnorderedPartitionSplitReader) splitReader) + long checkpointId, SplitReader, PulsarPartitionSplit> splitReader) { + return ((PulsarUnorderedPartitionSplitReader) splitReader) .snapshotState(checkpointId) .toPulsarPartitionSplit(); } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/message/PulsarMessage.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/message/PulsarMessage.java deleted file mode 100644 index 0632e2265d406..0000000000000 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/message/PulsarMessage.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.connector.pulsar.source.reader.message; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; - -import org.apache.pulsar.client.api.Message; -import org.apache.pulsar.client.api.MessageId; - -/** - * The message instance that contains the required information which would be used for committing - * the consuming status. - */ -@Internal -public class PulsarMessage { - - /** - * The id of a given message. This id could be same for multiple {@link PulsarMessage}, although - * it is unique for every {@link Message}. - */ - private final MessageId id; - - /** The value which deserialized by {@link PulsarDeserializationSchema}. */ - private final T value; - - /** The produce time for this message, it's a event time. */ - private final long eventTime; - - public PulsarMessage(MessageId id, T value, long eventTime) { - this.id = id; - this.value = value; - this.eventTime = eventTime; - } - - public MessageId getId() { - return id; - } - - public T getValue() { - return value; - } - - public long getEventTime() { - return eventTime; - } - - @Override - public String toString() { - return "PulsarMessage{" - + "id=" - + id - + ", value=" - + value - + ", eventTime=" - + eventTime - + '}'; - } -} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/message/PulsarMessageCollector.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/message/PulsarMessageCollector.java deleted file mode 100644 index f201425fc71cf..0000000000000 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/message/PulsarMessageCollector.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.connector.pulsar.source.reader.message; - -import org.apache.flink.connector.base.source.reader.RecordsBySplits; -import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; -import org.apache.flink.util.Collector; - -import org.apache.pulsar.client.api.Message; - -/** - * This collector supplier is providing the {@link Collector} for accepting the deserialized {@link - * PulsarMessage} from pulsar {@link PulsarDeserializationSchema}. - * - * @param The deserialized pulsar message type, aka the source message type. - */ -public class PulsarMessageCollector implements Collector { - - private final String splitId; - private final RecordsBySplits.Builder> builder; - private Message message; - - public PulsarMessageCollector( - String splitId, RecordsBySplits.Builder> builder) { - this.splitId = splitId; - this.builder = builder; - } - - public void setMessage(Message message) { - this.message = message; - } - - @Override - public void collect(T t) { - PulsarMessage result = - new PulsarMessage<>(message.getMessageId(), t, message.getEventTime()); - builder.add(splitId, result); - } - - @Override - public void close() { - // Nothing to do for this collector. - } -} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarOrderedSourceReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarOrderedSourceReader.java index 8c197afdf50c1..7280cd52c12f4 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarOrderedSourceReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarOrderedSourceReader.java @@ -26,14 +26,15 @@ import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; +import org.apache.flink.connector.pulsar.source.reader.emitter.PulsarRecordEmitter; import org.apache.flink.connector.pulsar.source.reader.fetcher.PulsarOrderedFetcherManager; -import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage; import org.apache.flink.connector.pulsar.source.reader.split.PulsarOrderedPartitionSplitReader; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplitState; import org.apache.flink.core.io.InputStatus; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClient; import org.slf4j.Logger; @@ -67,15 +68,17 @@ public class PulsarOrderedSourceReader extends PulsarSourceReaderBase private ScheduledExecutorService cursorScheduler; public PulsarOrderedSourceReader( - FutureCompletingBlockingQueue>> elementsQueue, - Supplier> splitReaderSupplier, + FutureCompletingBlockingQueue>> elementsQueue, + Supplier splitReaderSupplier, + PulsarRecordEmitter recordEmitter, SourceReaderContext context, SourceConfiguration sourceConfiguration, PulsarClient pulsarClient, PulsarAdmin pulsarAdmin) { super( elementsQueue, - new PulsarOrderedFetcherManager<>(elementsQueue, splitReaderSupplier::get), + new PulsarOrderedFetcherManager(elementsQueue, splitReaderSupplier::get), + recordEmitter, context, sourceConfiguration, pulsarClient, @@ -108,6 +111,9 @@ public InputStatus pollNext(ReaderOutput output) throws Exception { @Override protected void onSplitFinished(Map finishedSplitIds) { + // Close all the finished splits. + closeFinishedSplits(finishedSplitIds.keySet()); + // We don't require new splits, all the splits are pre-assigned by source enumerator. if (LOG.isDebugEnabled()) { LOG.debug("onSplitFinished event: {}", finishedSplitIds); @@ -147,7 +153,7 @@ public void notifyCheckpointComplete(long checkpointId) { LOG.debug("Committing cursors for checkpoint {}", checkpointId); Map cursors = cursorsToCommit.get(checkpointId); try { - ((PulsarOrderedFetcherManager) splitFetcherManager).acknowledgeMessages(cursors); + ((PulsarOrderedFetcherManager) splitFetcherManager).acknowledgeMessages(cursors); LOG.debug("Successfully acknowledge cursors for checkpoint {}", checkpointId); // Clean up the cursors. @@ -192,7 +198,7 @@ private void cumulativeAcknowledgmentMessage() { } try { - ((PulsarOrderedFetcherManager) splitFetcherManager).acknowledgeMessages(cursors); + ((PulsarOrderedFetcherManager) splitFetcherManager).acknowledgeMessages(cursors); // Clean up the finish splits. cursorsOfFinishedSplits.keySet().removeAll(cursors.keySet()); } catch (Exception e) { diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderBase.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderBase.java index 0122021145439..bf95ff858af18 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderBase.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderBase.java @@ -25,13 +25,15 @@ import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; import org.apache.flink.connector.pulsar.source.reader.emitter.PulsarRecordEmitter; import org.apache.flink.connector.pulsar.source.reader.fetcher.PulsarFetcherManagerBase; -import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplitState; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.PulsarClient; +import java.util.Set; + /** * The common pulsar source reader for both ordered & unordered message consuming. * @@ -39,25 +41,21 @@ */ abstract class PulsarSourceReaderBase extends SourceReaderBase< - PulsarMessage, OUT, PulsarPartitionSplit, PulsarPartitionSplitState> { + Message, OUT, PulsarPartitionSplit, PulsarPartitionSplitState> { protected final SourceConfiguration sourceConfiguration; protected final PulsarClient pulsarClient; protected final PulsarAdmin pulsarAdmin; protected PulsarSourceReaderBase( - FutureCompletingBlockingQueue>> elementsQueue, - PulsarFetcherManagerBase splitFetcherManager, + FutureCompletingBlockingQueue>> elementsQueue, + PulsarFetcherManagerBase splitFetcherManager, + PulsarRecordEmitter recordEmitter, SourceReaderContext context, SourceConfiguration sourceConfiguration, PulsarClient pulsarClient, PulsarAdmin pulsarAdmin) { - super( - elementsQueue, - splitFetcherManager, - new PulsarRecordEmitter<>(), - sourceConfiguration, - context); + super(elementsQueue, splitFetcherManager, recordEmitter, sourceConfiguration, context); this.sourceConfiguration = sourceConfiguration; this.pulsarClient = pulsarClient; @@ -75,6 +73,12 @@ protected PulsarPartitionSplit toSplitType( return splitState.toPulsarPartitionSplit(); } + protected void closeFinishedSplits(Set finishedSplitIds) { + for (String splitId : finishedSplitIds) { + ((PulsarFetcherManagerBase) splitFetcherManager).closeFetcher(splitId); + } + } + @Override public void close() throws Exception { // Close the all the consumers first. diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarUnorderedSourceReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarUnorderedSourceReader.java index 2af77d9a8467c..caa7199254139 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarUnorderedSourceReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarUnorderedSourceReader.java @@ -24,13 +24,14 @@ import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.reader.emitter.PulsarRecordEmitter; import org.apache.flink.connector.pulsar.source.reader.fetcher.PulsarUnorderedFetcherManager; -import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage; import org.apache.flink.connector.pulsar.source.reader.split.PulsarUnorderedPartitionSplitReader; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplitState; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient; import org.apache.pulsar.client.api.transaction.TxnID; @@ -47,6 +48,8 @@ import java.util.TreeMap; import java.util.function.Supplier; +import static java.util.stream.Collectors.toList; + /** * The source reader for pulsar subscription Shared and Key_Shared, which consumes the unordered * messages. @@ -60,8 +63,9 @@ public class PulsarUnorderedSourceReader extends PulsarSourceReaderBase transactionsOfFinishedSplits; public PulsarUnorderedSourceReader( - FutureCompletingBlockingQueue>> elementsQueue, - Supplier> splitReaderSupplier, + FutureCompletingBlockingQueue>> elementsQueue, + Supplier splitReaderSupplier, + PulsarRecordEmitter recordEmitter, SourceReaderContext context, SourceConfiguration sourceConfiguration, PulsarClient pulsarClient, @@ -69,7 +73,8 @@ public PulsarUnorderedSourceReader( @Nullable TransactionCoordinatorClient coordinatorClient) { super( elementsQueue, - new PulsarUnorderedFetcherManager<>(elementsQueue, splitReaderSupplier::get), + new PulsarUnorderedFetcherManager(elementsQueue, splitReaderSupplier::get), + recordEmitter, context, sourceConfiguration, pulsarClient, @@ -82,6 +87,9 @@ public PulsarUnorderedSourceReader( @Override protected void onSplitFinished(Map finishedSplitIds) { + // Close all the finished splits. + closeFinishedSplits(finishedSplitIds.keySet()); + // We don't require new splits, all the splits are pre-assigned by source enumerator. if (LOG.isDebugEnabled()) { LOG.debug("onSplitFinished event: {}", finishedSplitIds); @@ -103,8 +111,7 @@ protected void onSplitFinished(Map finishedSp public List snapshotState(long checkpointId) { LOG.debug("Trigger the new transaction for downstream readers."); List splits = - ((PulsarUnorderedFetcherManager) splitFetcherManager) - .snapshotState(checkpointId); + ((PulsarUnorderedFetcherManager) splitFetcherManager).snapshotState(checkpointId); if (coordinatorClient != null) { // Snapshot the transaction status and commit it after checkpoint finished. @@ -116,6 +123,11 @@ public List snapshotState(long checkpointId) { txnIDs.add(uncommittedTransactionId); } } + + // Add finished splits' transactions. + txnIDs.addAll(transactionsOfFinishedSplits); + // Purge the transactions. + transactionsOfFinishedSplits.clear(); } return splits; @@ -126,20 +138,38 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { LOG.debug("Committing transactions for checkpoint {}", checkpointId); if (coordinatorClient != null) { - for (Map.Entry> entry : transactionsToCommit.entrySet()) { - Long currentCheckpointId = entry.getKey(); - if (currentCheckpointId > checkpointId) { - continue; + List checkpointIds = + transactionsToCommit.keySet().stream() + .filter(id -> id <= checkpointId) + .collect(toList()); + + for (Long id : checkpointIds) { + List transactions = transactionsToCommit.remove(id); + if (transactions != null) { + for (TxnID transaction : transactions) { + coordinatorClient.commit(transaction); + } } + } + } + } - List transactions = entry.getValue(); + @Override + public void close() throws Exception { + // Abort all the pending transactions. + if (coordinatorClient != null) { + for (List transactions : transactionsToCommit.values()) { for (TxnID transaction : transactions) { - coordinatorClient.commit(transaction); - transactionsOfFinishedSplits.remove(transaction); + try { + coordinatorClient.abort(transaction); + } catch (Exception e) { + LOG.warn("Error in aborting transaction {}", transaction, e); + } } - - transactionsToCommit.remove(currentCheckpointId); } } + + // Close the pulsar client finally. + super.close(); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java index bb6d79641f503..200f5b356abc8 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java @@ -20,35 +20,37 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; -import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; import org.apache.flink.connector.pulsar.source.reader.source.PulsarOrderedSourceReader; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.pulsar.client.admin.PulsarAdmin; -import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.time.Duration; +import java.util.List; import java.util.concurrent.TimeUnit; import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; import static org.apache.flink.connector.pulsar.source.config.CursorVerification.FAIL_ON_MISMATCH; +import static org.apache.flink.connector.pulsar.source.enumerator.cursor.MessageIdUtils.nextMessageId; /** * The split reader a given {@link PulsarPartitionSplit}, it would be closed once the {@link * PulsarOrderedSourceReader} is closed. - * - * @param the type of the pulsar source message that would be serialized to downstream. */ @Internal -public class PulsarOrderedPartitionSplitReader extends PulsarPartitionSplitReaderBase { +public class PulsarOrderedPartitionSplitReader extends PulsarPartitionSplitReaderBase { private static final Logger LOG = LoggerFactory.getLogger(PulsarOrderedPartitionSplitReader.class); @@ -56,8 +58,9 @@ public PulsarOrderedPartitionSplitReader( PulsarClient pulsarClient, PulsarAdmin pulsarAdmin, SourceConfiguration sourceConfiguration, - PulsarDeserializationSchema deserializationSchema) { - super(pulsarClient, pulsarAdmin, sourceConfiguration, deserializationSchema); + Schema schema, + @Nullable CryptoKeyReader cryptoKeyReader) { + super(pulsarClient, pulsarAdmin, sourceConfiguration, schema, cryptoKeyReader); } @Override @@ -66,7 +69,7 @@ protected Message pollMessage(Duration timeout) throws PulsarClientExcep } @Override - protected void finishedPollMessage(Message message) { + protected void finishedPollMessage(Message message) { // Nothing to do here. LOG.debug("Finished polling message {}", message); @@ -75,18 +78,39 @@ protected void finishedPollMessage(Message message) { } @Override - protected void startConsumer(PulsarPartitionSplit split, Consumer consumer) { + protected void beforeCreatingConsumer(PulsarPartitionSplit split) { MessageId latestConsumedId = split.getLatestConsumedId(); // Reset the start position for ordered pulsar consumer. if (latestConsumedId != null) { - StartCursor startCursor = StartCursor.fromMessageId(latestConsumedId, false); - TopicPartition partition = split.getPartition(); - + LOG.info("Reset subscription position by the checkpoint {}", latestConsumedId); try { - startCursor.seekPosition( - partition.getTopic(), partition.getPartitionId(), consumer); - } catch (PulsarClientException e) { + MessageId initialPosition; + if (latestConsumedId == MessageId.latest + || latestConsumedId == MessageId.earliest) { + // for compatibility + initialPosition = latestConsumedId; + } else { + initialPosition = nextMessageId(latestConsumedId); + } + + // Remove Consumer.seek() here for waiting for pulsar-client-all 2.12.0 + // See https://github.com/apache/pulsar/issues/16757 for more details. + + String topicName = split.getPartition().getFullTopicName(); + List subscriptions = pulsarAdmin.topics().getSubscriptions(topicName); + String subscriptionName = sourceConfiguration.getSubscriptionName(); + + if (!subscriptions.contains(subscriptionName)) { + // If this subscription is not available. Just create it. + pulsarAdmin + .topics() + .createSubscription(topicName, subscriptionName, initialPosition); + } else { + // Reset the subscription if this is existed. + pulsarAdmin.topics().resetCursor(topicName, subscriptionName, initialPosition); + } + } catch (PulsarAdminException e) { if (sourceConfiguration.getVerifyInitialOffsets() == FAIL_ON_MISMATCH) { throw new IllegalArgumentException(e); } else { @@ -95,7 +119,7 @@ protected void startConsumer(PulsarPartitionSplit split, Consumer consum LOG.warn( "Failed to reset cursor to {} on partition {}", latestConsumedId, - partition, + split.getPartition(), e); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java index 37b5630a8d113..2e7b972d4472a 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java @@ -26,16 +26,15 @@ import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor.StopCondition; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; -import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; -import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage; -import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessageCollector; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.flink.util.Preconditions; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.ConsumerBuilder; +import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.KeySharedPolicy; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.PulsarClient; @@ -51,26 +50,20 @@ import java.time.Duration; import java.util.List; import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; import static org.apache.flink.connector.pulsar.source.config.PulsarSourceConfigUtils.createConsumerBuilder; -/** - * The common partition split reader. - * - * @param the type of the pulsar source message that would be serialized to downstream. - */ -abstract class PulsarPartitionSplitReaderBase - implements SplitReader, PulsarPartitionSplit> { +/** The common partition split reader. */ +abstract class PulsarPartitionSplitReaderBase + implements SplitReader, PulsarPartitionSplit> { private static final Logger LOG = LoggerFactory.getLogger(PulsarPartitionSplitReaderBase.class); protected final PulsarClient pulsarClient; protected final PulsarAdmin pulsarAdmin; protected final SourceConfiguration sourceConfiguration; - protected final PulsarDeserializationSchema deserializationSchema; - protected final AtomicBoolean wakeup; + protected final Schema schema; + @Nullable protected final CryptoKeyReader cryptoKeyReader; protected Consumer pulsarConsumer; protected PulsarPartitionSplit registeredSplit; @@ -79,60 +72,55 @@ protected PulsarPartitionSplitReaderBase( PulsarClient pulsarClient, PulsarAdmin pulsarAdmin, SourceConfiguration sourceConfiguration, - PulsarDeserializationSchema deserializationSchema) { + Schema schema, + @Nullable CryptoKeyReader cryptoKeyReader) { this.pulsarClient = pulsarClient; this.pulsarAdmin = pulsarAdmin; this.sourceConfiguration = sourceConfiguration; - this.deserializationSchema = deserializationSchema; - this.wakeup = new AtomicBoolean(false); + this.schema = schema; + this.cryptoKeyReader = cryptoKeyReader; } @Override - public RecordsWithSplitIds> fetch() throws IOException { - RecordsBySplits.Builder> builder = new RecordsBySplits.Builder<>(); + public RecordsWithSplitIds> fetch() throws IOException { + RecordsBySplits.Builder> builder = new RecordsBySplits.Builder<>(); // Return when no split registered to this reader. if (pulsarConsumer == null || registeredSplit == null) { return builder.build(); } - // Set wakeup to false for start consuming. - wakeup.compareAndSet(true, false); - StopCursor stopCursor = registeredSplit.getStopCursor(); String splitId = registeredSplit.splitId(); - PulsarMessageCollector collector = new PulsarMessageCollector<>(splitId, builder); Deadline deadline = Deadline.fromNow(sourceConfiguration.getMaxFetchTime()); - // Consume message from pulsar until it was woke up by flink reader. + // Consume messages from pulsar until it was waked up by flink reader. for (int messageNum = 0; - messageNum < sourceConfiguration.getMaxFetchRecords() - && deadline.hasTimeLeft() - && isNotWakeup(); + messageNum < sourceConfiguration.getMaxFetchRecords() && deadline.hasTimeLeft(); messageNum++) { try { - Duration timeout = deadline.timeLeftIfAny(); - Message message = pollMessage(timeout); + Message message = pollMessage(sourceConfiguration.getDefaultFetchTime()); if (message == null) { break; } - // Deserialize message. - collector.setMessage(message); - deserializationSchema.deserialize(message, collector); + StopCondition condition = stopCursor.shouldStop(message); - // Acknowledge message if need. - finishedPollMessage(message); + if (condition == StopCondition.CONTINUE || condition == StopCondition.EXACTLY) { + // Deserialize message. + builder.add(splitId, message); + + // Acknowledge message if needed. + finishedPollMessage(message); + } - if (stopCursor.shouldStop(message)) { + if (condition == StopCondition.EXACTLY || condition == StopCondition.TERMINATE) { builder.addFinishedSplit(splitId); break; } } catch (InterruptedException e) { Thread.currentThread().interrupt(); break; - } catch (TimeoutException e) { - break; } catch (ExecutionException e) { LOG.error("Error in polling message from pulsar consumer.", e); break; @@ -165,23 +153,27 @@ public void handleSplitsChanges(SplitsChange splitsChanges newSplits.size() == 1, "This pulsar split reader only support one split."); PulsarPartitionSplit newSplit = newSplits.get(0); + // Open stop cursor. + newSplit.open(pulsarAdmin); + + // Before creating the consumer. + beforeCreatingConsumer(newSplit); + // Create pulsar consumer. Consumer consumer = createPulsarConsumer(newSplit); - // Open start & stop cursor. - newSplit.open(pulsarAdmin); - - // Start Consumer. - startConsumer(newSplit, consumer); + // After creating the consumer. + afterCreatingConsumer(newSplit, consumer); LOG.info("Register split {} consumer for current reader.", newSplit); + this.registeredSplit = newSplit; this.pulsarConsumer = consumer; } @Override public void wakeUp() { - wakeup.compareAndSet(false, true); + // Nothing to do on this method. } @Override @@ -195,16 +187,18 @@ public void close() { protected abstract Message pollMessage(Duration timeout) throws ExecutionException, InterruptedException, PulsarClientException; - protected abstract void finishedPollMessage(Message message); - - protected abstract void startConsumer(PulsarPartitionSplit split, Consumer consumer); + protected abstract void finishedPollMessage(Message message); - // --------------------------- Helper Methods ----------------------------- + protected void beforeCreatingConsumer(PulsarPartitionSplit split) { + // Nothing to do by default. + } - protected boolean isNotWakeup() { - return !wakeup.get(); + protected void afterCreatingConsumer(PulsarPartitionSplit split, Consumer consumer) { + // Nothing to do by default. } + // --------------------------- Helper Methods ----------------------------- + /** Create a specified {@link Consumer} by the given split information. */ protected Consumer createPulsarConsumer(PulsarPartitionSplit split) { return createPulsarConsumer(split.getPartition()); @@ -213,10 +207,15 @@ protected Consumer createPulsarConsumer(PulsarPartitionSplit split) { /** Create a specified {@link Consumer} by the given topic partition. */ protected Consumer createPulsarConsumer(TopicPartition partition) { ConsumerBuilder consumerBuilder = - createConsumerBuilder(pulsarClient, Schema.BYTES, sourceConfiguration); + createConsumerBuilder(pulsarClient, schema, sourceConfiguration); consumerBuilder.topic(partition.getFullTopicName()); + // Add CryptoKeyReader if it exists for supporting end-to-end encryption. + if (cryptoKeyReader != null) { + consumerBuilder.cryptoKeyReader(cryptoKeyReader); + } + // Add KeySharedPolicy for Key_Shared subscription. if (sourceConfiguration.getSubscriptionType() == SubscriptionType.Key_Shared) { KeySharedPolicy policy = diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java index 5940cc9ac19be..b076557bb3a84 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java @@ -20,19 +20,21 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; -import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; import org.apache.flink.connector.pulsar.source.reader.source.PulsarUnorderedSourceReader; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplitState; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.transaction.Transaction; import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient; import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClientException; +import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClientException.TransactionNotFoundException; import org.apache.pulsar.client.api.transaction.TxnID; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,20 +47,17 @@ import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; import static org.apache.flink.connector.pulsar.common.utils.PulsarTransactionUtils.createTransaction; +import static org.apache.flink.connector.pulsar.common.utils.PulsarTransactionUtils.unwrap; /** * The split reader a given {@link PulsarPartitionSplit}, it would be closed once the {@link * PulsarUnorderedSourceReader} is closed. - * - * @param the type of the pulsar source message that would be serialized to downstream. */ @Internal -public class PulsarUnorderedPartitionSplitReader extends PulsarPartitionSplitReaderBase { +public class PulsarUnorderedPartitionSplitReader extends PulsarPartitionSplitReaderBase { private static final Logger LOG = LoggerFactory.getLogger(PulsarUnorderedPartitionSplitReader.class); - private static final Duration REDELIVER_TIME = Duration.ofSeconds(3); - private final TransactionCoordinatorClient coordinatorClient; @Nullable private Transaction uncommittedTransaction; @@ -67,9 +66,10 @@ public PulsarUnorderedPartitionSplitReader( PulsarClient pulsarClient, PulsarAdmin pulsarAdmin, SourceConfiguration sourceConfiguration, - PulsarDeserializationSchema deserializationSchema, + Schema schema, + @Nullable CryptoKeyReader cryptoKeyReader, TransactionCoordinatorClient coordinatorClient) { - super(pulsarClient, pulsarAdmin, sourceConfiguration, deserializationSchema); + super(pulsarClient, pulsarAdmin, sourceConfiguration, schema, cryptoKeyReader); this.coordinatorClient = coordinatorClient; } @@ -97,18 +97,8 @@ protected Message pollMessage(Duration timeout) .acknowledgeAsync(message.getMessageId(), uncommittedTransaction) .get(); } catch (InterruptedException e) { - sneakyClient( - () -> - pulsarConsumer.reconsumeLater( - message, REDELIVER_TIME.toMillis(), TimeUnit.MILLISECONDS)); Thread.currentThread().interrupt(); throw e; - } catch (ExecutionException e) { - sneakyClient( - () -> - pulsarConsumer.reconsumeLater( - message, REDELIVER_TIME.toMillis(), TimeUnit.MILLISECONDS)); - throw e; } } @@ -116,7 +106,7 @@ protected Message pollMessage(Duration timeout) } @Override - protected void finishedPollMessage(Message message) { + protected void finishedPollMessage(Message message) { if (sourceConfiguration.isEnableAutoAcknowledgeMessage()) { sneakyClient(() -> pulsarConsumer.acknowledge(message)); } @@ -126,7 +116,7 @@ protected void finishedPollMessage(Message message) { } @Override - protected void startConsumer(PulsarPartitionSplit split, Consumer consumer) { + protected void afterCreatingConsumer(PulsarPartitionSplit split, Consumer consumer) { TxnID uncommittedTransactionId = split.getUncommittedTransactionId(); // Abort the uncommitted pulsar transaction. @@ -135,10 +125,14 @@ protected void startConsumer(PulsarPartitionSplit split, Consumer consum try { coordinatorClient.abort(uncommittedTransactionId); } catch (TransactionCoordinatorClientException e) { - LOG.error( - "Failed to abort the uncommitted transaction {} when restart the reader", - uncommittedTransactionId, - e); + TransactionCoordinatorClientException exception = unwrap(e); + // The aborted transaction would return a not found exception. + if (!(exception instanceof TransactionNotFoundException)) { + LOG.error( + "Failed to abort the uncommitted transaction {} when restart the reader", + uncommittedTransactionId, + e); + } } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/split/PulsarPartitionSplit.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/split/PulsarPartitionSplit.java index 0ec693a2b26d2..2b2a7cda80fab 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/split/PulsarPartitionSplit.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/split/PulsarPartitionSplit.java @@ -30,14 +30,15 @@ import javax.annotation.Nullable; +import java.io.Serializable; import java.util.Objects; import static org.apache.flink.util.Preconditions.checkNotNull; /** A {@link SourceSplit} implementation for a Pulsar's partition. */ @Internal -public class PulsarPartitionSplit implements SourceSplit { - +public class PulsarPartitionSplit implements SourceSplit, Serializable { + private static final long serialVersionUID = -6857317360756062625L; private final TopicPartition partition; private final StopCursor stopCursor; @@ -60,8 +61,8 @@ public PulsarPartitionSplit(TopicPartition partition, StopCursor stopCursor) { public PulsarPartitionSplit( TopicPartition partition, StopCursor stopCursor, - MessageId latestConsumedId, - TxnID uncommittedTransactionId) { + @Nullable MessageId latestConsumedId, + @Nullable TxnID uncommittedTransactionId) { this.partition = checkNotNull(partition); this.stopCursor = checkNotNull(stopCursor); this.latestConsumedId = latestConsumedId; diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java new file mode 100644 index 0000000000000..b6ae8fc949869 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java @@ -0,0 +1,299 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.pulsar.common.config.PulsarOptions; +import org.apache.flink.connector.pulsar.sink.PulsarSinkOptions; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; +import org.apache.flink.connector.pulsar.source.PulsarSourceOptions; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.table.sink.PulsarTableSerializationSchemaFactory; +import org.apache.flink.connector.pulsar.table.sink.PulsarTableSink; +import org.apache.flink.connector.pulsar.table.source.PulsarTableDeserializationSchemaFactory; +import org.apache.flink.connector.pulsar.table.source.PulsarTableSource; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.DynamicTableSourceFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.types.DataType; + +import org.apache.pulsar.client.api.SubscriptionType; + +import java.util.List; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL; +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_SERVICE_URL; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.createKeyFormatProjection; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.createValueFormatProjection; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getKeyDecodingFormat; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getKeyEncodingFormat; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getMessageDelayMillis; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getPulsarProperties; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getStartCursor; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getStopCursor; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getSubscriptionType; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicListFromOptions; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicRouter; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicRoutingMode; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getValueDecodingFormat; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getValueEncodingFormat; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.ADMIN_URL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.EXPLICIT; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FORMAT; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SERVICE_URL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_CUSTOM_TOPIC_ROUTER; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_MESSAGE_DELAY_INTERVAL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AFTER_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_NAME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.VALUE_FORMAT; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validatePrimaryKeyConstraints; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateTableSinkOptions; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateTableSourceOptions; +import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM; +import static org.apache.pulsar.shade.org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; + +/** + * Factory for creating {@link DynamicTableSource} and {@link DynamicTableSink}. + * + *

The main role of this class is to retrieve config options and validate options from config and + * the table schema. It also sets default values if a config option is not present. + */ +public class PulsarTableFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory { + + public static final String IDENTIFIER = "pulsar"; + + public static final String DEFAULT_SUBSCRIPTION_NAME_PREFIX = "flink-sql-connector-pulsar-"; + + @Override + public DynamicTableSource createDynamicTableSource(Context context) { + FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); + // Format options should be retrieved before validation. + final DecodingFormat> keyDecodingFormat = + getKeyDecodingFormat(helper); + final DecodingFormat> valueDecodingFormat = + getValueDecodingFormat(helper); + ReadableConfig tableOptions = helper.getOptions(); + + // Validate configs are not conflict; each options is consumed; no unwanted configs + // PulsarOptions, PulsarSourceOptions and PulsarSinkOptions is not part of the validation. + helper.validateExcept( + PulsarOptions.CLIENT_CONFIG_PREFIX, + PulsarOptions.ADMIN_CONFIG_PREFIX, + PulsarSourceOptions.SOURCE_CONFIG_PREFIX, + PulsarSourceOptions.CONSUMER_CONFIG_PREFIX, + PulsarSinkOptions.PRODUCER_CONFIG_PREFIX, + PulsarSinkOptions.SINK_CONFIG_PREFIX); + + validatePrimaryKeyConstraints( + context.getObjectIdentifier(), context.getPrimaryKeyIndexes(), helper); + + validateTableSourceOptions(tableOptions); + + // Retrieve configs + final List topics = getTopicListFromOptions(tableOptions); + final StartCursor startCursor = getStartCursor(tableOptions); + final StopCursor stopCursor = getStopCursor(tableOptions); + final SubscriptionType subscriptionType = getSubscriptionType(tableOptions); + + // Forward source configs + final Properties properties = getPulsarProperties(tableOptions); + properties.setProperty(PULSAR_ADMIN_URL.key(), tableOptions.get(ADMIN_URL)); + properties.setProperty(PULSAR_SERVICE_URL.key(), tableOptions.get(SERVICE_URL)); + // Set random subscriptionName if not provided + properties.setProperty( + PULSAR_SUBSCRIPTION_NAME.key(), + tableOptions + .getOptional(SOURCE_SUBSCRIPTION_NAME) + .orElse(DEFAULT_SUBSCRIPTION_NAME_PREFIX + randomAlphabetic(5))); + // Retrieve physical fields (not including computed or metadata fields), + // and projections and create a schema factory based on such information. + final DataType physicalDataType = context.getPhysicalRowDataType(); + + final int[] valueProjection = createValueFormatProjection(tableOptions, physicalDataType); + final int[] keyProjection = createKeyFormatProjection(tableOptions, physicalDataType); + + final PulsarTableDeserializationSchemaFactory deserializationSchemaFactory = + new PulsarTableDeserializationSchemaFactory( + physicalDataType, + keyDecodingFormat, + keyProjection, + valueDecodingFormat, + valueProjection); + + // Set default values for configuration not exposed to user. + final DecodingFormat> decodingFormatForMetadataPushdown = + valueDecodingFormat; + + return new PulsarTableSource( + deserializationSchemaFactory, + decodingFormatForMetadataPushdown, + topics, + properties, + startCursor, + stopCursor, + subscriptionType); + } + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); + // Format options should be retrieved before validation. + final EncodingFormat> keyEncodingFormat = + getKeyEncodingFormat(helper); + final EncodingFormat> valueEncodingFormat = + getValueEncodingFormat(helper); + ReadableConfig tableOptions = helper.getOptions(); + + // Validate configs are not conflict; each options is consumed; no unwanted configs + // PulsarOptions, PulsarSourceOptions and PulsarSinkOptions is not part of the validation. + helper.validateExcept( + PulsarOptions.CLIENT_CONFIG_PREFIX, + PulsarOptions.ADMIN_CONFIG_PREFIX, + PulsarSourceOptions.SOURCE_CONFIG_PREFIX, + PulsarSourceOptions.CONSUMER_CONFIG_PREFIX, + PulsarSinkOptions.PRODUCER_CONFIG_PREFIX, + PulsarSinkOptions.SINK_CONFIG_PREFIX); + + validatePrimaryKeyConstraints( + context.getObjectIdentifier(), context.getPrimaryKeyIndexes(), helper); + + validateTableSinkOptions(tableOptions); + + // Retrieve configs + final TopicRouter topicRouter = + getTopicRouter(tableOptions, context.getClassLoader()); + final TopicRoutingMode topicRoutingMode = getTopicRoutingMode(tableOptions); + final long messageDelayMillis = getMessageDelayMillis(tableOptions); + + final List topics = getTopicListFromOptions(tableOptions); + + // Forward sink configs + final Properties properties = getPulsarProperties(tableOptions); + properties.setProperty(PULSAR_ADMIN_URL.key(), tableOptions.get(ADMIN_URL)); + properties.setProperty(PULSAR_SERVICE_URL.key(), tableOptions.get(SERVICE_URL)); + + // Retrieve physical DataType (not including computed or metadata fields) + final DataType physicalDataType = context.getPhysicalRowDataType(); + final int[] keyProjection = createKeyFormatProjection(tableOptions, physicalDataType); + final int[] valueProjection = createValueFormatProjection(tableOptions, physicalDataType); + + final PulsarTableSerializationSchemaFactory serializationSchemaFactory = + new PulsarTableSerializationSchemaFactory( + physicalDataType, + keyEncodingFormat, + keyProjection, + valueEncodingFormat, + valueProjection); + + // Set default values for configuration not exposed to user. + final DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE; + final ChangelogMode changelogMode = valueEncodingFormat.getChangelogMode(); + + return new PulsarTableSink( + serializationSchemaFactory, + changelogMode, + topics, + properties, + deliveryGuarantee, + topicRouter, + topicRoutingMode, + messageDelayMillis); + } + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + return Stream.of(TOPICS, ADMIN_URL, SERVICE_URL).collect(Collectors.toSet()); + } + + @Override + public Set> optionalOptions() { + return Stream.of( + FactoryUtil.FORMAT, + VALUE_FORMAT, + SOURCE_SUBSCRIPTION_NAME, + SOURCE_SUBSCRIPTION_TYPE, + SOURCE_START_FROM_MESSAGE_ID, + SOURCE_START_FROM_PUBLISH_TIME, + SOURCE_STOP_AT_MESSAGE_ID, + SOURCE_STOP_AFTER_MESSAGE_ID, + SOURCE_STOP_AT_PUBLISH_TIME, + SINK_CUSTOM_TOPIC_ROUTER, + SINK_TOPIC_ROUTING_MODE, + SINK_MESSAGE_DELAY_INTERVAL, + SINK_PARALLELISM, + KEY_FORMAT, + KEY_FIELDS, + EXPLICIT) + .collect(Collectors.toSet()); + } + + /** + * Format and Delivery guarantee related options are not forward options. + * + * @return + */ + @Override + public Set> forwardOptions() { + return Stream.of( + TOPICS, + ADMIN_URL, + SERVICE_URL, + SOURCE_SUBSCRIPTION_TYPE, + SOURCE_SUBSCRIPTION_NAME, + SOURCE_START_FROM_MESSAGE_ID, + SOURCE_START_FROM_PUBLISH_TIME, + SOURCE_STOP_AT_MESSAGE_ID, + SOURCE_STOP_AFTER_MESSAGE_ID, + SOURCE_STOP_AT_PUBLISH_TIME, + SINK_CUSTOM_TOPIC_ROUTER, + SINK_TOPIC_ROUTING_MODE, + SINK_MESSAGE_DELAY_INTERVAL) + .collect(Collectors.toSet()); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java new file mode 100644 index 0000000000000..8df1346d7f636 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java @@ -0,0 +1,317 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DeserializationFormatFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.InstantiationUtil; + +import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.impl.MessageIdImpl; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Properties; +import java.util.stream.IntStream; + +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FORMAT; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_CUSTOM_TOPIC_ROUTER; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_MESSAGE_DELAY_INTERVAL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AFTER_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.VALUE_FORMAT; +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * A util class for getting fields from config options, getting formats and other useful + * information. + * + *

It contains the following functionalities. + * + *

    + *
  • Get Topics from configurations. + *
  • Get source StartCursor from configurations. + *
  • Get source SubscriptionType from configurations. + *
  • Get sink messageDelayMillis from configurations. + *
  • Get sink TopicRouter/TopicRoutingMode from configurations. + *
  • Create key and value encoding/decoding format. + *
  • Create key and value projection. + *
+ */ +public class PulsarTableOptionUtils { + + private PulsarTableOptionUtils() {} + + // -------------------------------------------------------------------------------------------- + // Decoding / Encoding and Projection + // -------------------------------------------------------------------------------------------- + + @Nullable + public static DecodingFormat> getKeyDecodingFormat( + FactoryUtil.TableFactoryHelper helper) { + return helper.discoverOptionalDecodingFormat(DeserializationFormatFactory.class, KEY_FORMAT) + .orElse(null); + } + + @Nullable + public static EncodingFormat> getKeyEncodingFormat( + FactoryUtil.TableFactoryHelper helper) { + return helper.discoverOptionalEncodingFormat(SerializationFormatFactory.class, KEY_FORMAT) + .orElse(null); + } + + public static DecodingFormat> getValueDecodingFormat( + FactoryUtil.TableFactoryHelper helper) { + return helper.discoverOptionalDecodingFormat( + DeserializationFormatFactory.class, FactoryUtil.FORMAT) + .orElseGet( + () -> + helper.discoverDecodingFormat( + DeserializationFormatFactory.class, VALUE_FORMAT)); + } + + public static EncodingFormat> getValueEncodingFormat( + FactoryUtil.TableFactoryHelper helper) { + return helper.discoverOptionalEncodingFormat( + SerializationFormatFactory.class, FactoryUtil.FORMAT) + .orElseGet( + () -> + helper.discoverEncodingFormat( + SerializationFormatFactory.class, VALUE_FORMAT)); + } + + /** + * Creates an array of indices that determine which physical fields of the table schema to + * include in the key format and the order that those fields have in the key format. + */ + public static int[] createKeyFormatProjection( + ReadableConfig options, DataType physicalDataType) { + final LogicalType physicalType = physicalDataType.getLogicalType(); + checkArgument(physicalType.is(LogicalTypeRoot.ROW), "Row data type expected."); + final Optional optionalKeyFormat = options.getOptional(KEY_FORMAT); + final Optional> optionalKeyFields = options.getOptional(KEY_FIELDS); + + if (!optionalKeyFormat.isPresent()) { + return new int[0]; + } + + final List keyFields = optionalKeyFields.get(); + final List physicalFields = LogicalTypeChecks.getFieldNames(physicalType); + return keyFields.stream() + .mapToInt( + keyField -> { + final int pos = physicalFields.indexOf(keyField); + // check that field name exists + if (pos < 0) { + throw new ValidationException( + String.format( + "Could not find the field '%s' in the table schema for usage in the key format. " + + "A key field must be a regular, physical column. " + + "The following columns can be selected in the '%s' option: %s", + keyField, KEY_FIELDS.key(), physicalFields)); + } + // check that field name is prefixed correctly + return pos; + }) + .toArray(); + } + + public static int[] createValueFormatProjection( + ReadableConfig options, DataType physicalDataType) { + final LogicalType physicalType = physicalDataType.getLogicalType(); + checkArgument(physicalType.is(LogicalTypeRoot.ROW), "Row data type expected."); + + final int physicalFieldCount = LogicalTypeChecks.getFieldCount(physicalType); + final IntStream physicalFields = IntStream.range(0, physicalFieldCount); + final int[] keyProjection = createKeyFormatProjection(options, physicalDataType); + return physicalFields + .filter(pos -> IntStream.of(keyProjection).noneMatch(k -> k == pos)) + .toArray(); + } + + // -------------------------------------------------------------------------------------------- + // Table Source Option Utils + // -------------------------------------------------------------------------------------------- + + public static List getTopicListFromOptions(ReadableConfig tableOptions) { + List topics = tableOptions.get(TOPICS); + return topics; + } + + public static Properties getPulsarProperties(ReadableConfig tableOptions) { + final Properties pulsarProperties = new Properties(); + final Map configs = ((Configuration) tableOptions).toMap(); + configs.keySet().stream() + .filter(key -> key.startsWith("pulsar")) + .forEach(key -> pulsarProperties.put(key, configs.get(key))); + return pulsarProperties; + } + + public static Properties getPulsarProperties(Map configs) { + final Properties pulsarProperties = new Properties(); + configs.keySet().stream() + .filter(key -> key.startsWith("pulsar")) + .forEach(key -> pulsarProperties.put(key, configs.get(key))); + return pulsarProperties; + } + + public static StartCursor getStartCursor(ReadableConfig tableOptions) { + if (tableOptions.getOptional(SOURCE_START_FROM_MESSAGE_ID).isPresent()) { + return parseMessageIdStartCursor(tableOptions.get(SOURCE_START_FROM_MESSAGE_ID)); + } else if (tableOptions.getOptional(SOURCE_START_FROM_PUBLISH_TIME).isPresent()) { + return parsePublishTimeStartCursor(tableOptions.get(SOURCE_START_FROM_PUBLISH_TIME)); + } else { + return StartCursor.earliest(); + } + } + + public static StopCursor getStopCursor(ReadableConfig tableOptions) { + if (tableOptions.getOptional(SOURCE_STOP_AT_MESSAGE_ID).isPresent()) { + return parseAtMessageIdStopCursor(tableOptions.get(SOURCE_STOP_AT_MESSAGE_ID)); + } else if (tableOptions.getOptional(SOURCE_STOP_AFTER_MESSAGE_ID).isPresent()) { + return parseAfterMessageIdStopCursor(tableOptions.get(SOURCE_STOP_AFTER_MESSAGE_ID)); + } else if (tableOptions.getOptional(SOURCE_STOP_AT_PUBLISH_TIME).isPresent()) { + return parseAtPublishTimeStopCursor(tableOptions.get(SOURCE_STOP_AT_PUBLISH_TIME)); + } else { + return StopCursor.never(); + } + } + + public static SubscriptionType getSubscriptionType(ReadableConfig tableOptions) { + return tableOptions.get(SOURCE_SUBSCRIPTION_TYPE); + } + + protected static StartCursor parseMessageIdStartCursor(String config) { + if (Objects.equals(config, "earliest")) { + return StartCursor.earliest(); + } else if (Objects.equals(config, "latest")) { + return StartCursor.latest(); + } else { + return StartCursor.fromMessageId(parseMessageIdString(config)); + } + } + + protected static StartCursor parsePublishTimeStartCursor(Long config) { + return StartCursor.fromPublishTime(config); + } + + protected static StopCursor parseAtMessageIdStopCursor(String config) { + if (Objects.equals(config, "never")) { + return StopCursor.never(); + } else if (Objects.equals(config, "latest")) { + return StopCursor.latest(); + } else { + return StopCursor.atMessageId(parseMessageIdString(config)); + } + } + + protected static StopCursor parseAfterMessageIdStopCursor(String config) { + return StopCursor.afterMessageId(parseMessageIdString(config)); + } + + protected static StopCursor parseAtPublishTimeStopCursor(Long config) { + return StopCursor.atPublishTime(config); + } + + protected static MessageIdImpl parseMessageIdString(String config) { + String[] tokens = config.split(":", 3); + checkArgument(tokens.length == 3, "MessageId format must be ledgerId:entryId:partitionId."); + + try { + long ledgerId = Long.parseLong(tokens[0]); + long entryId = Long.parseLong(tokens[1]); + int partitionId = Integer.parseInt(tokens[2]); + MessageIdImpl messageId = new MessageIdImpl(ledgerId, entryId, partitionId); + return messageId; + } catch (NumberFormatException e) { + throw new IllegalArgumentException( + "MessageId format must be ledgerId:entryId:partitionId. " + + "Each id should be able to parsed to long type."); + } + } + + // -------------------------------------------------------------------------------------------- + // Table Sink Option Utils + // -------------------------------------------------------------------------------------------- + + public static TopicRouter getTopicRouter( + ReadableConfig readableConfig, ClassLoader classLoader) { + if (!readableConfig.getOptional(SINK_CUSTOM_TOPIC_ROUTER).isPresent()) { + return null; + } + + String className = readableConfig.get(SINK_CUSTOM_TOPIC_ROUTER); + try { + Class clazz = Class.forName(className, true, classLoader); + if (!TopicRouter.class.isAssignableFrom(clazz)) { + throw new ValidationException( + String.format( + "Sink TopicRouter class '%s' should extend from the required class %s", + className, TopicRouter.class.getName())); + } + @SuppressWarnings("unchecked") + final TopicRouter topicRouter = + InstantiationUtil.instantiate(className, TopicRouter.class, classLoader); + + return topicRouter; + } catch (ClassNotFoundException | FlinkException e) { + throw new ValidationException( + String.format( + "Could not find and instantiate TopicRouter class '%s'", className), + e); + } + } + + public static TopicRoutingMode getTopicRoutingMode(ReadableConfig readableConfig) { + return readableConfig.get(SINK_TOPIC_ROUTING_MODE); + } + + public static long getMessageDelayMillis(ReadableConfig readableConfig) { + return readableConfig.get(SINK_MESSAGE_DELAY_INTERVAL).toMillis(); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java new file mode 100644 index 0000000000000..2f3d9d602a3db --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java @@ -0,0 +1,282 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.description.Description; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; + +import org.apache.pulsar.client.api.SubscriptionType; + +import java.time.Duration; +import java.util.List; + +import static org.apache.flink.configuration.description.TextElement.code; +import static org.apache.flink.configuration.description.TextElement.text; +import static org.apache.flink.table.factories.FactoryUtil.FORMAT_SUFFIX; + +/** + * Config options that is used to configure a Pulsar SQL Connector. These config options are + * specific to SQL Connectors only. Other runtime configurations can be found in {@link + * org.apache.flink.connector.pulsar.common.config.PulsarOptions}, {@link + * org.apache.flink.connector.pulsar.source.PulsarSourceOptions}, and {@link + * org.apache.flink.connector.pulsar.sink.PulsarSinkOptions}. + */ +@PublicEvolving +public final class PulsarTableOptions { + + private PulsarTableOptions() {} + + public static final ConfigOption> TOPICS = + ConfigOptions.key("topics") + .stringType() + .asList() + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "Topic name(s) the table reads data from. It can be a single topic name or a list of topic names separated by a semicolon symbol (%s) like %s.", + code(";"), code("topic-1;topic-2")) + .build()); + + // -------------------------------------------------------------------------------------------- + // Table Source Options + // -------------------------------------------------------------------------------------------- + + public static final ConfigOption SOURCE_SUBSCRIPTION_TYPE = + ConfigOptions.key("source.subscription-type") + .enumType(SubscriptionType.class) + .defaultValue(SubscriptionType.Exclusive) + .withDescription( + Description.builder() + .text( + "The [subscription type](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-subscriptions) that is supported by the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source). Currently, only %s and %s subscription types are supported.", + code("Exclusive"), code("Shared")) + .build()); + + /** + * Exactly same as {@link + * org.apache.flink.connector.pulsar.source.PulsarSourceOptions#PULSAR_SUBSCRIPTION_NAME}. + * Copied because we want to have a default value for it. + */ + public static final ConfigOption SOURCE_SUBSCRIPTION_NAME = + ConfigOptions.key("source.subscription-name") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "The subscription name of the consumer that is used by the runtime [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source). This argument is required for constructing the consumer.") + .build()); + + public static final ConfigOption SOURCE_START_FROM_MESSAGE_ID = + ConfigOptions.key("source.start.message-id") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "Optional message id used to specify a consuming starting point for " + + "source. Use %s, %s or pass in a message id " + + "representation in %s, " + + "such as %s", + code("earliest"), + code("latest"), + code("ledgerId:entryId:partitionId"), + code("12:2:-1")) + .build()); + + public static final ConfigOption SOURCE_START_FROM_PUBLISH_TIME = + ConfigOptions.key("source.start.publish-time") + .longType() + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "(Optional) the publish timestamp that is used to specify a starting point for the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source) to consume data.") + .build()); + + public static final ConfigOption SOURCE_STOP_AT_MESSAGE_ID = + ConfigOptions.key("source.stop.at-message-id") + .stringType() + .noDefaultValue() + .withDescription( + "Optional message id used to specify a stop cursor for the unbounded sql " + + "source. Use \"never\", \"latest\" or pass in a message id " + + "representation in \"ledgerId:entryId:partitionId\", " + + "such as \"12:2:-1\""); + + public static final ConfigOption SOURCE_STOP_AFTER_MESSAGE_ID = + ConfigOptions.key("source.stop.after-message-id") + .stringType() + .noDefaultValue() + .withDescription( + "Optional message id used to specify a stop position but include the " + + "given message in the consuming result for the unbounded sql " + + "source. Pass in a message id " + + "representation in \"ledgerId:entryId:partitionId\", " + + "such as \"12:2:-1\". "); + + public static final ConfigOption SOURCE_STOP_AT_PUBLISH_TIME = + ConfigOptions.key("source.stop.at-publish-time") + .longType() + .noDefaultValue() + .withDescription( + "Optional publish timestamp used to specify a stop cursor" + + " for the unbounded sql source."); + + // -------------------------------------------------------------------------------------------- + // Table Sink Options + // -------------------------------------------------------------------------------------------- + + public static final ConfigOption SINK_CUSTOM_TOPIC_ROUTER = + ConfigOptions.key("sink.custom-topic-router") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "(Optional) the custom topic router class URL that is used in the [Pulsar DataStream sink connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-sink). If this option is provided, the %s option will be ignored.", + code("sink.topic-routing-mode")) + .build()); + + public static final ConfigOption SINK_TOPIC_ROUTING_MODE = + ConfigOptions.key("sink.topic-routing-mode") + .enumType(TopicRoutingMode.class) + .defaultValue(TopicRoutingMode.ROUND_ROBIN) + .withDescription( + Description.builder() + .text( + "(Optional) the topic routing mode. Available options are %s and %s. By default, it is set to %s. If you want to use a custom topic router, use the %s option to determine the partition for a particular message.", + code("round-robin"), + code("message-key-hash"), + code("round-robin"), + code("sink.custom-topic-router")) + .build()); + + public static final ConfigOption SINK_MESSAGE_DELAY_INTERVAL = + ConfigOptions.key("sink.message-delay-interval") + .durationType() + .defaultValue(Duration.ZERO) + .withDescription( + "(Optional) the message delay delivery interval that is used in the [Pulsar DataStream sink connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-sink)."); + + // -------------------------------------------------------------------------------------------- + // Format Options + // -------------------------------------------------------------------------------------------- + + public static final ConfigOption KEY_FORMAT = + ConfigOptions.key("key" + FORMAT_SUFFIX) + .stringType() + .noDefaultValue() + .withDescription( + "Defines the format identifier for decoding/encoding key bytes in " + + "Pulsar message. The identifier is used to discover a suitable format factory."); + + public static final ConfigOption> KEY_FIELDS = + ConfigOptions.key("key.fields") + .stringType() + .asList() + .defaultValues() + .withDescription( + "An explicit list of physical columns from the table schema that are decoded/encoded from the key bytes of a Pulsar message. By default, this list is empty and thus a key is undefined."); + + public static final ConfigOption VALUE_FORMAT = + ConfigOptions.key("value" + FORMAT_SUFFIX) + .stringType() + .noDefaultValue() + .withDescription( + "Defines the format identifier for decoding/encoding value data. " + + "The identifier is used to discover a suitable format factory."); + + // -------------------------------------------------------------------------------------------- + // Pulsar Options + // -------------------------------------------------------------------------------------------- + + /** + * Exactly same as {@link + * org.apache.flink.connector.pulsar.common.config.PulsarOptions#PULSAR_ADMIN_URL}. Copied here + * because it is a required config option and should not be included in the {@link + * org.apache.flink.table.factories.FactoryUtil.FactoryHelper#validateExcept(String...)} method. + * + *

By default all {@link org.apache.flink.connector.pulsar.common.config.PulsarOptions} are + * included in the validateExcept() method./p> + */ + public static final ConfigOption ADMIN_URL = + ConfigOptions.key("admin-url") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "The Pulsar service HTTP URL for the admin endpoint. For example, %s, or %s for TLS.", + code("http://my-broker.example.com:8080"), + code("https://my-broker.example.com:8443")) + .build()); + + /** + * Exactly same as {@link + * org.apache.flink.connector.pulsar.common.config.PulsarOptions#PULSAR_SERVICE_URL}. Copied + * here because it is a required config option and should not be included in the {@link + * org.apache.flink.table.factories.FactoryUtil.FactoryHelper#validateExcept(String...)} method. + * + *

By default all {@link org.apache.flink.connector.pulsar.common.config.PulsarOptions} are + * included in the validateExcept() method./p> + */ + public static final ConfigOption SERVICE_URL = + ConfigOptions.key("service-url") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text("Service URL provider for Pulsar service.") + .linebreak() + .text( + "To connect to Pulsar using client libraries, you need to specify a Pulsar protocol URL.") + .linebreak() + .text( + "You can assign Pulsar protocol URLs to specific clusters and use the Pulsar scheme.") + .linebreak() + .list( + text( + "This is an example of %s: %s.", + code("localhost"), + code("pulsar://localhost:6650")), + text( + "If you have multiple brokers, the URL is as: %s", + code( + "pulsar://localhost:6550,localhost:6651,localhost:6652")), + text( + "A URL for a production Pulsar cluster is as: %s", + code( + "pulsar://pulsar.us-west.example.com:6650")), + text( + "If you use TLS authentication, the URL is as %s", + code( + "pulsar+ssl://pulsar.us-west.example.com:6651"))) + .build()); + + public static final ConfigOption EXPLICIT = + ConfigOptions.key("explicit") + .booleanType() + .defaultValue(true) + .withDescription("Indicate if the table is an explicit Flink table."); +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java new file mode 100644 index 0000000000000..8d7e01825ca09 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java @@ -0,0 +1,182 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.types.RowKind; + +import org.apache.flink.shaded.guava30.com.google.common.collect.Sets; + +import org.apache.pulsar.client.api.SubscriptionType; + +import java.util.List; +import java.util.Optional; +import java.util.Set; + +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getValueDecodingFormat; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FORMAT; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_CUSTOM_TOPIC_ROUTER; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AFTER_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.pulsar.common.naming.TopicName.isValid; + +/** Util class for source and sink validation rules. */ +public class PulsarTableValidationUtils { + + private PulsarTableValidationUtils() {} + + public static void validatePrimaryKeyConstraints( + ObjectIdentifier tableName, + int[] primaryKeyIndexes, + FactoryUtil.TableFactoryHelper helper) { + final DecodingFormat> format = + getValueDecodingFormat(helper); + if (primaryKeyIndexes.length > 0 + && format.getChangelogMode().containsOnly(RowKind.INSERT)) { + throw new ValidationException( + String.format( + "The Pulsar table '%s' with '%s' format doesn't support defining PRIMARY KEY constraint" + + " on the table, because it can't guarantee the semantic of primary key.", + tableName.asSummaryString(), format)); + } + } + + public static void validateTableSourceOptions(ReadableConfig tableOptions) { + validateTopicsConfigs(tableOptions); + validateStartCursorConfigs(tableOptions); + validateStopCursorConfigs(tableOptions); + validateSubscriptionTypeConfigs(tableOptions); + validateKeyFormatConfigs(tableOptions); + } + + public static void validateTableSinkOptions(ReadableConfig tableOptions) { + validateTopicsConfigs(tableOptions); + validateKeyFormatConfigs(tableOptions); + validateSinkRoutingConfigs(tableOptions); + } + + protected static void validateTopicsConfigs(ReadableConfig tableOptions) { + if (tableOptions.get(TOPICS).isEmpty()) { + throw new ValidationException("The topics list should not be empty."); + } + + for (String topic : tableOptions.get(TOPICS)) { + if (!isValid(topic)) { + throw new ValidationException( + String.format("The topics name %s is not a valid topic name.", topic)); + } + } + } + + protected static void validateStartCursorConfigs(ReadableConfig tableOptions) { + if (tableOptions.getOptional(SOURCE_START_FROM_MESSAGE_ID).isPresent() + && tableOptions.getOptional(SOURCE_START_FROM_PUBLISH_TIME).isPresent()) { + throw new ValidationException( + String.format( + "Only one of %s and %s can be specified. Detected both of them", + SOURCE_START_FROM_MESSAGE_ID, SOURCE_START_FROM_PUBLISH_TIME)); + } + } + + protected static void validateStopCursorConfigs(ReadableConfig tableOptions) { + Set> conflictConfigOptions = + Sets.newHashSet( + SOURCE_STOP_AT_MESSAGE_ID, + SOURCE_STOP_AFTER_MESSAGE_ID, + SOURCE_STOP_AT_PUBLISH_TIME); + + long configsNums = + conflictConfigOptions.stream() + .map(tableOptions::getOptional) + .filter(Optional::isPresent) + .count(); + + if (configsNums > 1) { + throw new ValidationException( + String.format( + "Only one of %s, %s and %s can be specified. Detected more than 1 of them", + SOURCE_STOP_AT_MESSAGE_ID, + SOURCE_STOP_AFTER_MESSAGE_ID, + SOURCE_STOP_AT_PUBLISH_TIME)); + } + } + + protected static void validateSubscriptionTypeConfigs(ReadableConfig tableOptions) { + SubscriptionType subscriptionType = tableOptions.get(SOURCE_SUBSCRIPTION_TYPE); + if (subscriptionType == SubscriptionType.Failover + || subscriptionType == SubscriptionType.Key_Shared) { + throw new ValidationException( + String.format( + "Only %s and %s SubscriptionType is supported. ", + SubscriptionType.Exclusive, SubscriptionType.Shared)); + } + } + + protected static void validateKeyFormatConfigs(ReadableConfig tableOptions) { + final Optional optionalKeyFormat = tableOptions.getOptional(KEY_FORMAT); + final Optional> optionalKeyFields = tableOptions.getOptional(KEY_FIELDS); + if (!optionalKeyFormat.isPresent() && optionalKeyFields.isPresent()) { + throw new ValidationException( + String.format( + "The option '%s' can only be declared if a key format is defined using '%s'.", + KEY_FIELDS.key(), KEY_FORMAT.key())); + } else if (optionalKeyFormat.isPresent() + && (!optionalKeyFields.isPresent() || optionalKeyFields.get().size() == 0)) { + throw new ValidationException( + String.format( + "A key format '%s' requires the declaration of one or more of key fields using '%s'.", + KEY_FORMAT.key(), KEY_FIELDS.key())); + } + } + + protected static void validateSinkRoutingConfigs(ReadableConfig tableOptions) { + if (tableOptions.getOptional(SINK_TOPIC_ROUTING_MODE).orElse(TopicRoutingMode.ROUND_ROBIN) + == TopicRoutingMode.CUSTOM) { + throw new ValidationException( + String.format( + "Only %s and %s can be used. For %s, please use sink.custom-topic-router for" + + "custom topic router and do not set this config.", + TopicRoutingMode.ROUND_ROBIN, + TopicRoutingMode.MESSAGE_KEY_HASH, + TopicRoutingMode.CUSTOM)); + } + if (tableOptions.getOptional(SINK_CUSTOM_TOPIC_ROUTER).isPresent() + && tableOptions.getOptional(SINK_TOPIC_ROUTING_MODE).isPresent()) { + throw new ValidationException( + String.format( + "Only one of %s and %s can be specified. Detected both of them", + SINK_CUSTOM_TOPIC_ROUTER, SINK_TOPIC_ROUTING_MODE)); + } + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalog.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalog.java new file mode 100644 index 0000000000000..1ff1bb0353d2f --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalog.java @@ -0,0 +1,460 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.catalog; + +import org.apache.flink.connector.pulsar.table.PulsarTableFactory; +import org.apache.flink.connector.pulsar.table.catalog.impl.IncompatibleSchemaException; +import org.apache.flink.connector.pulsar.table.catalog.impl.PulsarCatalogSupport; +import org.apache.flink.connector.pulsar.table.catalog.impl.SchemaTranslator; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.CatalogDatabaseImpl; +import org.apache.flink.table.catalog.CatalogPartition; +import org.apache.flink.table.catalog.CatalogPartitionSpec; +import org.apache.flink.table.catalog.GenericInMemoryCatalog; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.ResolvedCatalogView; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.PartitionAlreadyExistsException; +import org.apache.flink.table.catalog.exceptions.PartitionNotExistException; +import org.apache.flink.table.catalog.exceptions.PartitionSpecInvalidException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; +import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; +import org.apache.flink.table.catalog.stats.CatalogTableStatistics; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.factories.Factory; + +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.List; +import java.util.Optional; + +import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogFactoryOptions.DEFAULT_DATABASE; + +/** + * Catalog implementation to use Pulsar to store metadatas for Flink tables/databases. + * + *

A {@link PulsarCatalog} offers two modes when mapping a Pulsar topic to a Flink table. + * + *

explicit table: an explict table refers to a table created using CREATE statements. In this + * mode, users are allowed to create a table that is bind to an existing Pulsar topic. Users can + * specify watermarks, metatdata fields utilize the verbose configuration options to customize the + * table connector. + * + *

native table: an native table refers to a table created by the Catalog and not by users using + * a 1-to-1 mapping from Flink table to Pulsar topic. Each existing Pulsar topic will be mapped to a + * table under a database using the topic's tenant and namespace named like 'tenant/namespace'. The + * mapped table has the same name as the Pulsar topic. This mode allows users to easily query from + * existing Pulsar topics without explicitly create the table. It automatically determines the Flink + * format to use based on the stored Pulsar schema in the Pulsar topic. This mode has some + * limitations, such as users can't designate an watermark and thus can't use window aggregation + * functions. + * + *

Each topic(except Pulsar system topics) is mapped to a native table, and users can create + * arbitrary number of explicit tables that binds to one Pulsar topic besides the native table. + */ +public class PulsarCatalog extends GenericInMemoryCatalog { + private static final Logger LOG = LoggerFactory.getLogger(PulsarCatalog.class); + + private final PulsarCatalogConfiguration catalogConfiguration; + + private PulsarCatalogSupport catalogSupport; + + private final String flinkTenant; + + public static final String DEFAULT_TENANT = "__flink_catalog"; + + public static final String DEFAULT_DB = "default_database"; + + public PulsarCatalog( + String catalogName, + PulsarCatalogConfiguration catalogConfiguration, + String database, + String flinkTenant) { + super(catalogName, database); + this.catalogConfiguration = catalogConfiguration; + this.flinkTenant = flinkTenant; + } + + @Override + public Optional getFactory() { + return Optional.of(new PulsarTableFactory()); + } + + @Override + public void open() throws CatalogException { + if (catalogSupport == null) { + try { + catalogSupport = + new PulsarCatalogSupport( + catalogConfiguration, flinkTenant, new SchemaTranslator(false)); + } catch (PulsarAdminException e) { + throw new CatalogException( + "Failed to create Pulsar admin with configuration:" + + catalogConfiguration.toString(), + e); + } + } + + CatalogDatabaseImpl defaultDatabase = + new CatalogDatabaseImpl(new HashMap<>(), "The default database for PulsarCatalog"); + try { + createDatabase(catalogConfiguration.get(DEFAULT_DATABASE), defaultDatabase, true); + } catch (DatabaseAlreadyExistException e) { + throw new CatalogException( + "Error: should ignore default database if not exist instead of throwing exception"); + } + } + + @Override + public void close() throws CatalogException { + if (catalogSupport != null) { + catalogSupport.close(); + catalogSupport = null; + LOG.info("Closed connection to Pulsar."); + } + } + + @Override + public List listDatabases() throws CatalogException { + try { + return catalogSupport.listDatabases(); + } catch (PulsarAdminException e) { + throw new CatalogException( + String.format("Failed to list all databases in catalog: %s", getName()), e); + } + } + + @Override + public CatalogDatabase getDatabase(String databaseName) throws CatalogException { + try { + return catalogSupport.getDatabase(databaseName); + } catch (PulsarAdminException e) { + throw new CatalogException( + String.format("Failed to get database info in catalog: %s", getName()), e); + } + } + + @Override + public boolean databaseExists(String databaseName) throws CatalogException { + try { + return catalogSupport.databaseExists(databaseName); + } catch (PulsarAdminException e) { + LOG.warn("Failed to check if database exists, encountered PulsarAdminError", e); + return false; + } catch (Exception e) { + LOG.error("Failed to check if database exists", e); + return false; + } + } + + @Override + public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists) + throws DatabaseAlreadyExistException, CatalogException { + try { + catalogSupport.createDatabase(name, database); + } catch (PulsarAdminException.ConflictException e) { + if (!ignoreIfExists) { + throw new DatabaseAlreadyExistException(getName(), name, e); + } + } catch (PulsarAdminException e) { + throw new CatalogException(String.format("Failed to create database %s", name), e); + } + } + + @Override + public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) + throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException { + try { + if (!listTables(name).isEmpty() && !cascade) { + throw new DatabaseNotEmptyException(getName(), name); + } + + // the cascade deletion relies on the pulsar namespace deletion will clear all topics + catalogSupport.dropDatabase(name); + } catch (PulsarAdminException.NotFoundException e) { + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), name); + } + } catch (PulsarAdminException e) { + throw new CatalogException(String.format("Failed to drop database %s", name), e); + } + } + + @Override + public List listTables(String databaseName) + throws DatabaseNotExistException, CatalogException { + try { + return catalogSupport.listTables(databaseName); + } catch (PulsarAdminException.NotFoundException e) { + throw new DatabaseNotExistException(getName(), databaseName, e); + } catch (PulsarAdminException e) { + throw new CatalogException( + String.format("Failed to list tables in database %s", databaseName), e); + } + } + + @Override + public CatalogBaseTable getTable(ObjectPath tablePath) + throws TableNotExistException, CatalogException { + if (tablePath.getObjectName().startsWith("_tmp_table_")) { + return super.getTable(tablePath); + } + try { + return catalogSupport.getTable(tablePath); + } catch (PulsarAdminException.NotFoundException e) { + throw new TableNotExistException(getName(), tablePath, e); + } catch (PulsarAdminException | IncompatibleSchemaException e) { + throw new CatalogException( + String.format("Failed to get table %s schema", tablePath.getFullName()), e); + } + } + + @Override + public boolean tableExists(ObjectPath tablePath) throws CatalogException { + if (tablePath.getObjectName().startsWith("_tmp_table_")) { + return super.tableExists(tablePath); + } + try { + return catalogSupport.tableExists(tablePath); + } catch (PulsarAdminException.NotFoundException e) { + return false; + } catch (PulsarAdminException e) { + throw new CatalogException( + String.format("Failed to check table %s existence", tablePath.getFullName()), + e); + } + } + + @Override + public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists) + throws TableAlreadyExistException, DatabaseNotExistException, CatalogException { + if (tablePath.getObjectName().startsWith("_tmp_table_")) { + super.createTable(tablePath, table, ignoreIfExists); + } + + if (!databaseExists(tablePath.getDatabaseName())) { + throw new DatabaseNotExistException(getName(), tablePath.getDatabaseName()); + } + + if (tableExists(tablePath)) { + if (!ignoreIfExists) { + throw new TableAlreadyExistException(getName(), tablePath); + } else { + return; + } + } + + if (table instanceof ResolvedCatalogTable) { + try { + catalogSupport.createTable(tablePath, (ResolvedCatalogTable) table); + } catch (PulsarAdminException | IncompatibleSchemaException e) { + throw new CatalogException( + String.format("Failed to create table %s", tablePath.getFullName()), e); + } + } else if (table instanceof ResolvedCatalogView) { + throw new CatalogException( + String.format( + "Can't create view %s with catalog %s", + tablePath.getObjectName(), getName())); + } else { + throw new CatalogException( + String.format("Unknown Table Object kind: %s", table.getClass().getName())); + } + } + + @Override + public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + try { + catalogSupport.dropTable(tablePath); + } catch (PulsarAdminException.NotFoundException e) { + if (!ignoreIfNotExists) { + throw new TableNotExistException(getName(), tablePath, e); + } else { + LOG.warn("The table {} does not exist. Drop table operation ignored", tablePath); + } + } catch (PulsarAdminException | RuntimeException e) { + throw new CatalogException( + String.format("Failed to drop table %s", tablePath.getFullName()), e); + } + } + + // ------------------------------------------------------------------------ + // Unsupported catalog operations for Pulsar + // There should not be such permission in the connector, it is very dangerous + // ------------------------------------------------------------------------ + + @Override + public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists) + throws DatabaseNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public List listViews(String databaseName) + throws DatabaseNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterTable( + ObjectPath tablePath, CatalogBaseTable newTable, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists) + throws TableNotExistException, TableAlreadyExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public List listPartitions(ObjectPath tablePath) + throws TableNotExistException, TableNotPartitionedException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public List listPartitions( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws TableNotExistException, TableNotPartitionedException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public List listPartitionsByFilter( + ObjectPath tablePath, List expressions) + throws TableNotExistException, TableNotPartitionedException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public CatalogPartition getPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws PartitionNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public boolean partitionExists(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void createPartition( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogPartition partition, + boolean ignoreIfExists) + throws TableNotExistException, TableNotPartitionedException, + PartitionSpecInvalidException, PartitionAlreadyExistsException, + CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void dropPartition( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec, boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterPartition( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogPartition newPartition, + boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public CatalogTableStatistics getTableStatistics(ObjectPath tablePath) + throws TableNotExistException, CatalogException { + return CatalogTableStatistics.UNKNOWN; + } + + @Override + public CatalogColumnStatistics getTableColumnStatistics(ObjectPath tablePath) + throws TableNotExistException, CatalogException { + return CatalogColumnStatistics.UNKNOWN; + } + + @Override + public CatalogTableStatistics getPartitionStatistics( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws PartitionNotExistException, CatalogException { + return CatalogTableStatistics.UNKNOWN; + } + + @Override + public CatalogColumnStatistics getPartitionColumnStatistics( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws PartitionNotExistException, CatalogException { + return CatalogColumnStatistics.UNKNOWN; + } + + @Override + public void alterTableStatistics( + ObjectPath tablePath, CatalogTableStatistics tableStatistics, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterTableColumnStatistics( + ObjectPath tablePath, + CatalogColumnStatistics columnStatistics, + boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterPartitionStatistics( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogTableStatistics partitionStatistics, + boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterPartitionColumnStatistics( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogColumnStatistics columnStatistics, + boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogConfiguration.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogConfiguration.java new file mode 100644 index 0000000000000..30cf3482acc8e --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogConfiguration.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.catalog; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.pulsar.common.config.PulsarConfiguration; + +/** The configuration class for {@link PulsarCatalog}. */ +public class PulsarCatalogConfiguration extends PulsarConfiguration { + private static final long serialVersionUID = 3139935676757015589L; + + /** + * Creates a new PulsarConfiguration, which holds a copy of the given configuration that can't + * be altered. PulsarCatalogConfiguration does not have extra configs besides {@link + * PulsarConfiguration} + * + * @param config The configuration with the original contents. + */ + public PulsarCatalogConfiguration(Configuration config) { + super(config); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogFactory.java new file mode 100644 index 0000000000000..6b4b091b0973d --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogFactory.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.catalog; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.pulsar.common.config.PulsarConfigBuilder; +import org.apache.flink.connector.pulsar.common.config.PulsarConfigValidator; +import org.apache.flink.connector.pulsar.common.config.PulsarOptions; +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.factories.CatalogFactory; +import org.apache.flink.table.factories.FactoryUtil; + +import java.util.Collections; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogFactoryOptions.AUTH_PARAMS; +import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogFactoryOptions.AUTH_PLUGIN; +import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogFactoryOptions.CATALOG_ADMIN_URL; +import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogFactoryOptions.CATALOG_SERVICE_URL; +import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogFactoryOptions.CATALOG_TENANT; +import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogFactoryOptions.DEFAULT_DATABASE; + +/** PulsarCatalogFactory implementing {@link CatalogFactory}. */ +public class PulsarCatalogFactory implements CatalogFactory { + + public static final String IDENTIFIER = "pulsar-catalog"; + + public static final PulsarConfigValidator CATALOG_CONFIG_VALIDATOR = + PulsarConfigValidator.builder().build(); + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Catalog createCatalog(Context context) { + final FactoryUtil.CatalogFactoryHelper helper = + FactoryUtil.createCatalogFactoryHelper(this, context); + + helper.validate(); + + PulsarConfigBuilder configBuilder = new PulsarConfigBuilder(); + ReadableConfig tableOptions = helper.getOptions(); + + configBuilder.set(PulsarOptions.PULSAR_ADMIN_URL, tableOptions.get(CATALOG_ADMIN_URL)); + configBuilder.set(PulsarOptions.PULSAR_SERVICE_URL, tableOptions.get(CATALOG_SERVICE_URL)); + if (tableOptions.getOptional(AUTH_PLUGIN).isPresent()) { + configBuilder.set( + PulsarOptions.PULSAR_AUTH_PLUGIN_CLASS_NAME, tableOptions.get(AUTH_PLUGIN)); + } + + if (tableOptions.getOptional(AUTH_PARAMS).isPresent()) { + configBuilder.set(PulsarOptions.PULSAR_AUTH_PARAMS, tableOptions.get(AUTH_PARAMS)); + } + + PulsarCatalogConfiguration catalogConfiguration = + configBuilder.build(CATALOG_CONFIG_VALIDATOR, PulsarCatalogConfiguration::new); + + return new PulsarCatalog( + context.getName(), + catalogConfiguration, + helper.getOptions().get(DEFAULT_DATABASE), + helper.getOptions().get(CATALOG_TENANT)); + } + + @Override + public Set> requiredOptions() { + return Collections.emptySet(); + } + + @Override + public Set> optionalOptions() { + // pulsar catalog options + return Stream.of( + CATALOG_ADMIN_URL, + CATALOG_SERVICE_URL, + CATALOG_TENANT, + DEFAULT_DATABASE, + AUTH_PLUGIN, + AUTH_PARAMS) + .collect(Collectors.toSet()); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogFactoryOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogFactoryOptions.java new file mode 100644 index 0000000000000..6ee4f3e66dc83 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogFactoryOptions.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.catalog; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.table.catalog.CommonCatalogOptions; + +/** {@link ConfigOption}s for {@link PulsarCatalog}. */ +@PublicEvolving +public final class PulsarCatalogFactoryOptions { + public static final ConfigOption CATALOG_TENANT = + ConfigOptions.key("catalog-tenant") + .stringType() + .defaultValue(PulsarCatalog.DEFAULT_TENANT) + .withDescription("The Pulsar tenant that stores all table information."); + + public static final ConfigOption DEFAULT_DATABASE = + ConfigOptions.key(CommonCatalogOptions.DEFAULT_DATABASE_KEY) + .stringType() + .defaultValue(PulsarCatalog.DEFAULT_DB) + .withDescription( + "The default database when using PulsarCatalog. It will be created if it does not exist."); + + public static final ConfigOption CATALOG_ADMIN_URL = + ConfigOptions.key("catalog-admin-url") + .stringType() + .defaultValue("http://localhost:8080") + .withDescription("(Required) The admin URL of the Pulsar cluster."); + + public static final ConfigOption CATALOG_SERVICE_URL = + ConfigOptions.key("catalog-service-url") + .stringType() + .defaultValue("pulsar://localhost:6650") + .withDescription("(Required) The service URL of the Pulsar cluster."); + + public static final ConfigOption AUTH_PLUGIN = + ConfigOptions.key("catalog-auth-plugin") + .stringType() + .noDefaultValue() + .withDescription( + "The name of the authentication plugin for accessing the Pulsar cluster."); + + public static final ConfigOption AUTH_PARAMS = + ConfigOptions.key("catalog-auth-params") + .stringType() + .noDefaultValue() + .withDescription( + "The authentication parameters for accessing the Pulsar cluster."); + + private PulsarCatalogFactoryOptions() {} +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/IncompatibleSchemaException.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/IncompatibleSchemaException.java new file mode 100644 index 0000000000000..2816e65f13512 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/IncompatibleSchemaException.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.catalog.impl; + +/** Exception designates the incompatibility between pulsar and flink type. */ +public class IncompatibleSchemaException extends IllegalStateException { + private static final long serialVersionUID = -2073484724665131680L; + + public IncompatibleSchemaException(String message, Throwable e) { + super(message, e); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/PulsarAdminTool.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/PulsarAdminTool.java new file mode 100644 index 0000000000000..f70a2f5c2f379 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/PulsarAdminTool.java @@ -0,0 +1,201 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.catalog.impl; + +import org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogConfiguration; +import org.apache.flink.connector.pulsar.table.catalog.utils.TableSchemaHelper; + +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.impl.schema.BytesSchema; +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.schema.SchemaInfo; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.connector.pulsar.common.config.PulsarClientFactory.createAdmin; + +/** A class that wraps Pulsar Admin API. */ +public class PulsarAdminTool implements AutoCloseable { + + // system topics are not filtered out by default in Pulsar 2.10.0 + // this filter is incomplete and should be replaced by SystemTopicNames class + // after 2.10.1 released. + private static final String SYSTEM_TOPIC_PREFIX = "__"; + + private final PulsarAdmin admin; + + public PulsarAdminTool(PulsarCatalogConfiguration catalogConfiguration) { + this.admin = createAdmin(catalogConfiguration); + } + + @Override + public void close() { + admin.close(); + } + + public void createTenant(String tenant) throws PulsarAdminException { + Set clusters = new HashSet<>(admin.clusters().getClusters()); + admin.tenants() + .createTenant(tenant, TenantInfoImpl.builder().allowedClusters(clusters).build()); + } + + public boolean tenantExists(String tenant) throws PulsarAdminException { + try { + admin.tenants().getTenantInfo(tenant); + } catch (PulsarAdminException.NotFoundException e) { + return false; + } + return true; + } + + public List listNamespaces() throws PulsarAdminException { + List tenants = admin.tenants().getTenants(); + List namespaces = new ArrayList(); + for (String tenant : tenants) { + namespaces.addAll(admin.namespaces().getNamespaces(tenant)); + } + return namespaces; + } + + public boolean namespaceExists(String ns) throws PulsarAdminException { + try { + admin.namespaces().getTopics(ns); + } catch (PulsarAdminException.NotFoundException e) { + return false; + } + return true; + } + + public void createNamespace(String ns) throws PulsarAdminException { + createNamespace(ns, false); + } + + public void createNamespace(String ns, boolean retain) throws PulsarAdminException { + String nsName = NamespaceName.get(ns).toString(); + admin.namespaces().createNamespace(nsName); + if (retain) { + // retain the topic infinitely to store the metadata + admin.namespaces().setRetention(nsName, new RetentionPolicies(-1, -1)); + } + } + + public void updateNamespaceProperties(String ns, Map properties) + throws PulsarAdminException { + admin.namespaces().setProperties(ns, properties); + } + + public Map getNamespaceProperties(String ns) throws PulsarAdminException { + return admin.namespaces().getProperties(ns); + } + + public void deleteNamespace(String ns) throws PulsarAdminException { + String nsName = NamespaceName.get(ns).toString(); + admin.namespaces().deleteNamespace(nsName); + } + + public List getTopics(String ns) throws PulsarAdminException { + List nonPartitionedTopics = getNonPartitionedTopics(ns); + List partitionedTopics = admin.topics().getPartitionedTopicList(ns); + List allTopics = new ArrayList<>(); + Stream.of(partitionedTopics, nonPartitionedTopics).forEach(allTopics::addAll); + return allTopics.stream() + .map(t -> TopicName.get(t).getLocalName()) + .filter(topic -> !topic.startsWith(SYSTEM_TOPIC_PREFIX)) + .collect(Collectors.toList()); + } + + public boolean topicExists(String topicName) throws PulsarAdminException { + try { + PartitionedTopicMetadata partitionedTopicMetadata = + admin.topics().getPartitionedTopicMetadata(topicName); + if (partitionedTopicMetadata.partitions > 0) { + return true; + } + } catch (PulsarAdminException.NotFoundException e) { + return false; + } + return false; + } + + public void deleteTopic(String topicName) throws PulsarAdminException { + + try { + admin.topics().deletePartitionedTopic(topicName, true); + } catch (PulsarAdminException.NotFoundException e) { + admin.topics().delete(topicName, true); + } + } + + public void createTopic(String topicName, int partitionNum) + throws PulsarAdminException, IncompatibleSchemaException { + if (partitionNum > 0) { + admin.topics().createPartitionedTopic(topicName, partitionNum); + } else { + admin.topics().createNonPartitionedTopic(topicName); + } + } + + public void uploadSchema(String topicName, SchemaInfo schemaInfo) + throws IncompatibleSchemaException { + TableSchemaHelper.uploadPulsarSchema(admin, topicName, schemaInfo); + } + + public void deleteSchema(String topicName) { + TableSchemaHelper.deletePulsarSchema(admin, topicName); + } + + public SchemaInfo getPulsarSchema(String topic) { + try { + return admin.schemas().getSchemaInfo(TopicName.get(topic).toString()); + } catch (PulsarAdminException e) { + if (e.getStatusCode() == 404) { + return BytesSchema.of().getSchemaInfo(); + } else { + throw new IllegalStateException( + String.format( + "Failed to get schema information for %s", + TopicName.get(topic).toString()), + e); + } + } catch (Throwable e) { + throw new IllegalStateException( + String.format( + "Failed to get schema information for %s", + TopicName.get(topic).toString()), + e); + } + } + + private List getNonPartitionedTopics(String namespace) throws PulsarAdminException { + return admin.topics().getList(namespace).stream() + .filter(t -> !TopicName.get(t).isPartitioned()) + .collect(Collectors.toList()); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/PulsarCatalogSupport.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/PulsarCatalogSupport.java new file mode 100644 index 0000000000000..9395f3bea3e5b --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/PulsarCatalogSupport.java @@ -0,0 +1,316 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.catalog.impl; + +import org.apache.flink.connector.pulsar.common.config.PulsarOptions; +import org.apache.flink.connector.pulsar.table.PulsarTableFactory; +import org.apache.flink.connector.pulsar.table.PulsarTableOptions; +import org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogConfiguration; +import org.apache.flink.connector.pulsar.table.catalog.utils.TableSchemaHelper; +import org.apache.flink.formats.raw.RawFormatFactory; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.CatalogDatabaseImpl; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.factories.FactoryUtil; + +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.TopicDomain; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.schema.SchemaInfo; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * This class is the implementation layer of catalog operations. It uses {@link PulsarAdminTool} to + * interact with Pulsar topics and manipulates metadata. {@link PulsarCatalogSupport} distinguish + * between explicit and native tables. + */ +public class PulsarCatalogSupport { + + private static final String DATABASE_COMMENT_KEY = "__database_comment"; + private static final String DATABASE_DESCRIPTION_KEY = "__database_description"; + private static final String DATABASE_DETAILED_DESCRIPTION_KEY = + "__database_detailed_description"; + + private static final String TABLE_PREFIX = "table_"; + + PulsarCatalogConfiguration catalogConfiguration; + + private final PulsarAdminTool pulsarAdminTool; + + private final String flinkCatalogTenant; + + private SchemaTranslator schemaTranslator; + + public PulsarCatalogSupport( + PulsarCatalogConfiguration catalogConfiguration, + String flinkTenant, + SchemaTranslator schemaTranslator) + throws PulsarAdminException { + this.catalogConfiguration = catalogConfiguration; + this.pulsarAdminTool = new PulsarAdminTool(catalogConfiguration); + this.schemaTranslator = schemaTranslator; + this.flinkCatalogTenant = flinkTenant; + + // Initialize the dedicated tenant if necessary + if (!pulsarAdminTool.tenantExists(flinkCatalogTenant)) { + pulsarAdminTool.createTenant(flinkCatalogTenant); + } + } + + /** + * A generic database stored in pulsar catalog should consist of alphanumeric characters. A + * pulsar tenant/namespace mapped database should contain the "/" in between tenant and + * namespace + * + * @param name the database name + * @return false if the name contains "/", which indicate it's a pulsar tenant/namespace mapped + * database + */ + private boolean isExplicitDatabase(String name) { + return !name.contains("/"); + } + + private String completeExplicitDatabasePath(String name) { + return this.flinkCatalogTenant + "/" + name; + } + + public List listDatabases() throws PulsarAdminException { + List databases = new ArrayList<>(); + for (String ns : pulsarAdminTool.listNamespaces()) { + if (ns.startsWith(flinkCatalogTenant)) { + // explicit table database + databases.add(ns.substring(flinkCatalogTenant.length() + 1)); + } else { + // pulsar tenant/namespace mapped database + databases.add(ns); + } + } + return databases; + } + + public boolean databaseExists(String name) throws PulsarAdminException { + if (isExplicitDatabase(name)) { + return pulsarAdminTool.namespaceExists(completeExplicitDatabasePath(name)); + } else { + return pulsarAdminTool.namespaceExists(name); + } + } + + public void createDatabase(String name, CatalogDatabase database) throws PulsarAdminException { + if (isExplicitDatabase(name)) { + pulsarAdminTool.createNamespace(completeExplicitDatabasePath(name)); + Map allProperties = database.getProperties(); + allProperties.put(DATABASE_COMMENT_KEY, database.getComment()); + allProperties.put(DATABASE_DESCRIPTION_KEY, database.getDescription().orElse("")); + allProperties.put( + DATABASE_DETAILED_DESCRIPTION_KEY, + database.getDetailedDescription().orElse("")); + pulsarAdminTool.updateNamespaceProperties( + completeExplicitDatabasePath(name), allProperties); + } else { + throw new CatalogException("Can't create pulsar tenant/namespace mapped database"); + } + } + + public CatalogDatabase getDatabase(String name) throws PulsarAdminException { + Map allProperties = + pulsarAdminTool.getNamespaceProperties(completeExplicitDatabasePath(name)); + String comment = allProperties.getOrDefault(DATABASE_COMMENT_KEY, ""); + allProperties.remove(DATABASE_COMMENT_KEY); + return new CatalogDatabaseImpl(allProperties, comment); + } + + public void dropDatabase(String name) throws PulsarAdminException { + if (isExplicitDatabase(name)) { + pulsarAdminTool.deleteNamespace(completeExplicitDatabasePath(name)); + } else { + throw new CatalogException("Can't drop pulsar tenant/namespace mapped database"); + } + } + + public List listTables(String name) throws PulsarAdminException { + if (isExplicitDatabase(name)) { + List tables = new ArrayList<>(); + List topics = pulsarAdminTool.getTopics(completeExplicitDatabasePath(name)); + for (String topic : topics) { + tables.add(topic.substring(TABLE_PREFIX.length())); + } + return tables; + } else { + return pulsarAdminTool.getTopics(name); + } + } + + public boolean tableExists(ObjectPath tablePath) throws PulsarAdminException { + if (isExplicitDatabase(tablePath.getDatabaseName())) { + return pulsarAdminTool.topicExists(findExplicitTablePlaceholderTopic(tablePath)); + } else { + return pulsarAdminTool.topicExists(findTopicForNativeTable(tablePath)); + } + } + + public CatalogTable getTable(ObjectPath tablePath) throws PulsarAdminException { + if (isExplicitDatabase(tablePath.getDatabaseName())) { + try { + String mappedTopic = findExplicitTablePlaceholderTopic(tablePath); + final SchemaInfo metadataSchema = pulsarAdminTool.getPulsarSchema(mappedTopic); + Map tableProperties = + TableSchemaHelper.generateTableProperties(metadataSchema); + CatalogTable table = CatalogTable.fromProperties(tableProperties); + table.getOptions().put(PulsarTableOptions.EXPLICIT.key(), Boolean.TRUE.toString()); + return CatalogTable.of( + table.getUnresolvedSchema(), + table.getComment(), + table.getPartitionKeys(), + fillDefaultOptionsFromCatalogOptions(table.getOptions())); + } catch (Exception e) { + e.printStackTrace(); + throw new CatalogException( + "Failed to fetch metadata for explict table: " + tablePath.getObjectName()); + } + } else { + String existingTopic = findTopicForNativeTable(tablePath); + final SchemaInfo pulsarSchema = pulsarAdminTool.getPulsarSchema(existingTopic); + return schemaToCatalogTable(pulsarSchema, existingTopic); + } + } + + public void dropTable(ObjectPath tablePath) throws PulsarAdminException { + if (isExplicitDatabase(tablePath.getDatabaseName())) { + String mappedTopic = findExplicitTablePlaceholderTopic(tablePath); + // manually clean the schema to avoid affecting new table with same name use old schema + pulsarAdminTool.deleteSchema(mappedTopic); + pulsarAdminTool.deleteTopic(mappedTopic); + } else { + throw new CatalogException("Can't delete native topic"); + } + } + + public void createTable(ObjectPath tablePath, ResolvedCatalogTable table) + throws PulsarAdminException { + // only allow creating table in explict database, the topic is used to save table + // information + if (!isExplicitDatabase(tablePath.getDatabaseName())) { + throw new CatalogException( + String.format( + "Can't create explict table under pulsar tenant/namespace: %s because it's a native database", + tablePath.getDatabaseName())); + } + + String mappedTopic = findExplicitTablePlaceholderTopic(tablePath); + pulsarAdminTool.createTopic(mappedTopic, 1); + + // use pulsar schema to store explicit table information + try { + SchemaInfo schemaInfo = TableSchemaHelper.generateSchemaInfo(table.toProperties()); + pulsarAdminTool.uploadSchema(mappedTopic, schemaInfo); + } catch (Exception e) { + // delete topic if table info cannot be persisted + try { + pulsarAdminTool.deleteTopic(mappedTopic); + } catch (PulsarAdminException ex) { + // do nothing + } + e.printStackTrace(); + throw new CatalogException("Can't store table metadata"); + } + } + + private CatalogTable schemaToCatalogTable(SchemaInfo pulsarSchema, String topicName) { + final Schema schema = schemaTranslator.pulsarSchemaToFlinkSchema(pulsarSchema); + + Map initialTableOptions = new HashMap<>(); + initialTableOptions.put(PulsarTableOptions.TOPICS.key(), topicName); + initialTableOptions.put( + FactoryUtil.FORMAT.key(), schemaTranslator.decideDefaultFlinkFormat(pulsarSchema)); + + Map enrichedTableOptions = + fillDefaultOptionsFromCatalogOptions(initialTableOptions); + + return CatalogTable.of(schema, "", Collections.emptyList(), enrichedTableOptions); + } + + // enrich table properties with proper catalog configs + private Map fillDefaultOptionsFromCatalogOptions( + final Map tableOptions) { + Map enrichedTableOptions = new HashMap<>(); + enrichedTableOptions.put(FactoryUtil.CONNECTOR.key(), PulsarTableFactory.IDENTIFIER); + enrichedTableOptions.put( + PulsarTableOptions.ADMIN_URL.key(), + catalogConfiguration.get(PulsarOptions.PULSAR_ADMIN_URL)); + enrichedTableOptions.put( + PulsarTableOptions.SERVICE_URL.key(), + catalogConfiguration.get(PulsarOptions.PULSAR_SERVICE_URL)); + + String authPlugin = catalogConfiguration.get(PulsarOptions.PULSAR_AUTH_PLUGIN_CLASS_NAME); + if (authPlugin != null && !authPlugin.isEmpty()) { + enrichedTableOptions.put(PulsarOptions.PULSAR_AUTH_PLUGIN_CLASS_NAME.key(), authPlugin); + } + + String authParams = catalogConfiguration.get(PulsarOptions.PULSAR_AUTH_PARAMS); + if (authParams != null && !authParams.isEmpty()) { + enrichedTableOptions.put(PulsarOptions.PULSAR_AUTH_PARAMS.key(), authParams); + } + + // we always provide RAW format as a default format + if (!enrichedTableOptions.containsKey(FactoryUtil.FORMAT.key())) { + enrichedTableOptions.put(FactoryUtil.FORMAT.key(), RawFormatFactory.IDENTIFIER); + } + + if (tableOptions != null) { + // table options could overwrite the default options provided above + enrichedTableOptions.putAll(tableOptions); + } + return enrichedTableOptions; + } + + private String findExplicitTablePlaceholderTopic(ObjectPath objectPath) { + String database = flinkCatalogTenant + "/" + objectPath.getDatabaseName(); + String topic = TABLE_PREFIX + objectPath.getObjectName(); + + NamespaceName ns = NamespaceName.get(database); + TopicName fullName = TopicName.get(TopicDomain.persistent.toString(), ns, topic); + return fullName.toString(); + } + + private String findTopicForNativeTable(ObjectPath objectPath) { + String database = objectPath.getDatabaseName(); + String topic = objectPath.getObjectName(); + + NamespaceName ns = NamespaceName.get(database); + TopicName fullName = TopicName.get(TopicDomain.persistent.toString(), ns, topic); + return fullName.toString(); + } + + public void close() { + if (pulsarAdminTool != null) { + pulsarAdminTool.close(); + } + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/SchemaTranslator.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/SchemaTranslator.java new file mode 100644 index 0000000000000..b5df200a5b93c --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/SchemaTranslator.java @@ -0,0 +1,239 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.catalog.impl; + +import org.apache.flink.formats.avro.AvroFormatFactory; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.formats.json.JsonFormatFactory; +import org.apache.flink.formats.raw.RawFormatFactory; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.FieldsDataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; + +import com.google.protobuf.Descriptors; +import org.apache.pulsar.client.impl.schema.generic.GenericProtobufNativeSchema; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; + +/** Translate a Pulsar Schema to Flink Table Schema. */ +public class SchemaTranslator { + private static final Logger LOG = LoggerFactory.getLogger(SchemaTranslator.class); + + public static final String SINGLE_FIELD_FIELD_NAME = "value"; + + private final boolean useMetadataFields; + + public SchemaTranslator(boolean useMetadataFields) { + this.useMetadataFields = useMetadataFields; + } + + public org.apache.flink.table.api.Schema pulsarSchemaToFlinkSchema(SchemaInfo pulsarSchema) + throws IncompatibleSchemaException { + final DataType fieldsDataType = pulsarSchemaToPhysicalFields(pulsarSchema); + org.apache.flink.table.api.Schema.Builder schemaBuilder = + org.apache.flink.table.api.Schema.newBuilder().fromRowDataType(fieldsDataType); + + if (useMetadataFields) { + throw new UnsupportedOperationException( + "Querying Pulsar Metadata is not supported yet"); + } + + return schemaBuilder.build(); + } + + public DataType pulsarSchemaToPhysicalFields(SchemaInfo schemaInfo) + throws IncompatibleSchemaException { + List mainSchema = new ArrayList<>(); + DataType dataType = schemaInfo2SqlType(schemaInfo); + // ROW and STRUCTURED are FieldsDataType + if (dataType instanceof FieldsDataType) { + FieldsDataType fieldsDataType = (FieldsDataType) dataType; + RowType rowType = (RowType) fieldsDataType.getLogicalType(); + List fieldNames = rowType.getFieldNames(); + for (int i = 0; i < fieldNames.size(); i++) { + org.apache.flink.table.types.logical.LogicalType logicalType = rowType.getTypeAt(i); + DataTypes.Field field = + DataTypes.FIELD( + fieldNames.get(i), + TypeConversions.fromLogicalToDataType(logicalType)); + mainSchema.add(field); + } + + } else { + mainSchema.add(DataTypes.FIELD(SINGLE_FIELD_FIELD_NAME, dataType)); + } + + return DataTypes.ROW(mainSchema.toArray(new DataTypes.Field[0])); + } + + public DataType schemaInfo2SqlType(SchemaInfo si) throws IncompatibleSchemaException { + switch (si.getType()) { + case NONE: + case BYTES: + return DataTypes.BYTES(); + case BOOLEAN: + return DataTypes.BOOLEAN(); + case LOCAL_DATE: + return DataTypes.DATE(); + case LOCAL_TIME: + return DataTypes.TIME(); + case STRING: + return DataTypes.STRING(); + case LOCAL_DATE_TIME: + return DataTypes.TIMESTAMP(3); + case INT8: + return DataTypes.TINYINT(); + case DOUBLE: + return DataTypes.DOUBLE(); + case FLOAT: + return DataTypes.FLOAT(); + case INT32: + return DataTypes.INT(); + case INT64: + return DataTypes.BIGINT(); + case INT16: + return DataTypes.SMALLINT(); + case AVRO: + case JSON: + String avroSchemaString = new String(si.getSchema(), StandardCharsets.UTF_8); + return AvroSchemaConverter.convertToDataType(avroSchemaString); + case PROTOBUF_NATIVE: + Descriptors.Descriptor descriptor = + ((GenericProtobufNativeSchema) GenericProtobufNativeSchema.of(si)) + .getProtobufNativeSchema(); + return protoDescriptorToSqlType(descriptor); + + default: + throw new UnsupportedOperationException( + String.format("We do not support %s currently.", si.getType())); + } + } + + public static DataType protoDescriptorToSqlType(Descriptors.Descriptor descriptor) + throws IncompatibleSchemaException { + List fields = new ArrayList<>(); + List protoFields = descriptor.getFields(); + + for (Descriptors.FieldDescriptor fieldDescriptor : protoFields) { + DataType fieldType = protoFieldDescriptorToSqlType(fieldDescriptor); + fields.add(DataTypes.FIELD(fieldDescriptor.getName(), fieldType)); + } + + if (fields.isEmpty()) { + throw new IllegalArgumentException("No FieldDescriptors found"); + } + return DataTypes.ROW(fields.toArray(new DataTypes.Field[0])); + } + + private static DataType protoFieldDescriptorToSqlType(Descriptors.FieldDescriptor field) + throws IncompatibleSchemaException { + Descriptors.FieldDescriptor.JavaType type = field.getJavaType(); + DataType dataType; + switch (type) { + case BOOLEAN: + dataType = DataTypes.BOOLEAN(); + break; + case BYTE_STRING: + dataType = DataTypes.BYTES(); + break; + case DOUBLE: + dataType = DataTypes.DOUBLE(); + break; + case ENUM: + dataType = DataTypes.STRING(); + break; + case FLOAT: + dataType = DataTypes.FLOAT(); + break; + case INT: + dataType = DataTypes.INT(); + break; + case LONG: + dataType = DataTypes.BIGINT(); + break; + case MESSAGE: + Descriptors.Descriptor msg = field.getMessageType(); + if (field.isMapField()) { + // map + dataType = + DataTypes.MAP( + protoFieldDescriptorToSqlType(msg.findFieldByName("key")), + protoFieldDescriptorToSqlType(msg.findFieldByName("value"))); + } else { + // row + dataType = protoDescriptorToSqlType(field.getMessageType()); + } + break; + case STRING: + dataType = DataTypes.STRING(); + break; + default: + throw new IllegalArgumentException( + "Unknown type: " + + type.toString() + + " for FieldDescriptor: " + + field.toString()); + } + // list + if (field.isRepeated() && !field.isMapField()) { + dataType = DataTypes.ARRAY(dataType); + } + + return dataType; + } + + /** + * This method is used to determine the Flink format to use for a native table. + * + * @param pulsarSchemaInfo + * @return + */ + public String decideDefaultFlinkFormat(SchemaInfo pulsarSchemaInfo) { + String formatIdentifier = RawFormatFactory.IDENTIFIER; + switch (pulsarSchemaInfo.getType()) { + case JSON: + formatIdentifier = JsonFormatFactory.IDENTIFIER; + break; + case AVRO: + formatIdentifier = AvroFormatFactory.IDENTIFIER; + break; + case PROTOBUF_NATIVE: + case PROTOBUF: + case AUTO_CONSUME: + case AUTO: + case AUTO_PUBLISH: + LOG.error( + "Can't decide format for {} schema", pulsarSchemaInfo.getType().toString()); + throw new UnsupportedOperationException( + String.format( + "Can't decide format for %s schema", + pulsarSchemaInfo.getType().toString())); + default: + break; + } + return formatIdentifier; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/utils/TableSchemaHelper.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/utils/TableSchemaHelper.java new file mode 100644 index 0000000000000..9961665d2aefa --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/utils/TableSchemaHelper.java @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.catalog.utils; + +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.table.types.DataType; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.type.TypeReference; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.impl.schema.SchemaInfoImpl; +import org.apache.pulsar.client.internal.DefaultImplementation; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.protocol.schema.PostSchemaPayload; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.schema.SchemaType; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.avro.Schema.Type.RECORD; +import static org.apache.pulsar.shade.com.google.common.base.Preconditions.checkNotNull; + +/** Util to convert between flink table map representation and pulsar SchemaInfo. */ +public final class TableSchemaHelper { + + private TableSchemaHelper() {} + + public static SchemaInfo generateSchemaInfo(Map properties) + throws JsonProcessingException { + ObjectMapper mapper = new ObjectMapper(); + return SchemaInfoImpl.builder() + .name("flink_table_schema") + .type(SchemaType.BYTES) + .schema(mapper.writeValueAsBytes(properties)) + .build(); + } + + public static Map generateTableProperties(SchemaInfo schemaInfo) + throws IOException { + ObjectMapper mapper = new ObjectMapper(); + return mapper.readValue( + schemaInfo.getSchema(), new TypeReference>() {}); + } + + public static void uploadPulsarSchema(PulsarAdmin admin, String topic, SchemaInfo schemaInfo) { + checkNotNull(schemaInfo); + + SchemaInfo existingSchema; + try { + existingSchema = admin.schemas().getSchemaInfo(TopicName.get(topic).toString()); + } catch (PulsarAdminException pae) { + if (pae.getStatusCode() == 404) { + existingSchema = null; + } else { + throw new RuntimeException( + String.format( + "Failed to get schema information for %s", + TopicName.get(topic).toString()), + pae); + } + } catch (Throwable e) { + throw new RuntimeException( + String.format( + "Failed to get schema information for %s", + TopicName.get(topic).toString()), + e); + } + + if (existingSchema == null) { + PostSchemaPayload pl = new PostSchemaPayload(); + try { + pl.setType(schemaInfo.getType().name()); + pl.setSchema(getSchemaString(schemaInfo)); + pl.setProperties(schemaInfo.getProperties()); + admin.schemas().createSchema(TopicName.get(topic).toString(), pl); + } catch (PulsarAdminException pae) { + if (pae.getStatusCode() == 404) { + throw new RuntimeException( + String.format( + "Create schema for %s get 404", + TopicName.get(topic).toString()), + pae); + } else { + throw new RuntimeException( + String.format( + "Failed to create schema information for %s", + TopicName.get(topic).toString()), + pae); + } + } catch (IOException e) { + throw new RuntimeException( + String.format( + "Failed to set schema information for %s", + TopicName.get(topic).toString()), + e); + } catch (Throwable e) { + throw new RuntimeException( + String.format( + "Failed to create schema information for %s", + TopicName.get(topic).toString()), + e); + } + } else if (!schemaEqualsIgnoreProperties(schemaInfo, existingSchema) + && !compatibleSchema(existingSchema, schemaInfo)) { + throw new RuntimeException("Writing to a topic which have incompatible schema"); + } + } + + public static void deletePulsarSchema(PulsarAdmin admin, String topic) { + try { + admin.schemas().deleteSchema(topic); + } catch (PulsarAdminException e) { + e.printStackTrace(); + } + } + + private static boolean schemaEqualsIgnoreProperties( + SchemaInfo schemaInfo, SchemaInfo existingSchema) { + return existingSchema.getType().equals(schemaInfo.getType()) + && Arrays.equals(existingSchema.getSchema(), schemaInfo.getSchema()); + } + + // TODO handle the exception + private static String getSchemaString(SchemaInfo schemaInfo) throws IOException { + final byte[] schemaData = schemaInfo.getSchema(); + if (null == schemaData) { + return null; + } + if (schemaInfo.getType() == SchemaType.KEY_VALUE) { + return DefaultImplementation.getDefaultImplementation() + .convertKeyValueSchemaInfoDataToString( + DefaultImplementation.getDefaultImplementation() + .decodeKeyValueSchemaInfo(schemaInfo)); + } + return new String(schemaData, StandardCharsets.UTF_8); + } + + public static boolean compatibleSchema(SchemaInfo s1, SchemaInfo s2) { + if (s1.getType() == SchemaType.NONE && s2.getType() == SchemaType.BYTES) { + return true; + } else { + return s1.getType() == SchemaType.BYTES && s2.getType() == SchemaType.NONE; + } + } + + public static SchemaInfoImpl getSchemaInfo(SchemaType type, DataType dataType) { + byte[] schemaBytes = getAvroSchema(dataType).toString().getBytes(StandardCharsets.UTF_8); + return SchemaInfoImpl.builder() + .name("Record") + .schema(schemaBytes) + .type(type) + .properties(Collections.emptyMap()) + .build(); + } + + public static org.apache.avro.Schema getAvroSchema(DataType dataType) { + org.apache.avro.Schema schema = + AvroSchemaConverter.convertToSchema(dataType.getLogicalType()); + if (schema.isNullable()) { + schema = + schema.getTypes().stream() + .filter(s -> s.getType() == RECORD) + .findAny() + .orElseThrow( + () -> + new IllegalArgumentException( + "not support DataType: " + + dataType.toString())); + } + return schema; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java new file mode 100644 index 0000000000000..cc54cc2773454 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java @@ -0,0 +1,171 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.sink; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.connector.pulsar.common.schema.BytesSchema; +import org.apache.flink.connector.pulsar.common.schema.PulsarSchema; +import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; +import org.apache.flink.connector.pulsar.sink.writer.context.PulsarSinkContext; +import org.apache.flink.connector.pulsar.sink.writer.message.PulsarMessage; +import org.apache.flink.connector.pulsar.sink.writer.message.PulsarMessageBuilder; +import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema; +import org.apache.flink.formats.avro.AvroRowDataSerializationSchema; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.formats.json.JsonRowDataSerializationSchema; +import org.apache.flink.formats.raw.RawFormatSerializationSchema; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RawType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; + +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.schema.SchemaDefinition; + +import javax.annotation.Nullable; + +import java.io.Serializable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link PulsarSerializationSchema} implementation for Pulsar SQL sink connector. It is + * responsible for retrieving fields from Flink row and serialize into Pulsar message key or body, + * and set necessary metadata fields as required. + */ +public class PulsarTableSerializationSchema implements PulsarSerializationSchema { + + private static final long serialVersionUID = 7314442107082067836L; + + @Nullable private final SerializationSchema keySerialization; + + private final RowData.FieldGetter[] keyFieldGetters; + + private final SerializationSchema valueSerialization; + + private final RowData.FieldGetter[] valueFieldGetters; + + private final PulsarWritableMetadata writableMetadata; + + private final RowType rowType; + + public PulsarTableSerializationSchema( + @Nullable SerializationSchema keySerialization, + RowData.FieldGetter[] keyFieldGetters, + SerializationSchema valueSerialization, + RowData.FieldGetter[] valueFieldGetters, + PulsarWritableMetadata writableMetadata, + RowType rowType) { + this.keySerialization = keySerialization; + this.keyFieldGetters = checkNotNull(keyFieldGetters); + this.valueSerialization = checkNotNull(valueSerialization); + this.valueFieldGetters = checkNotNull(valueFieldGetters); + this.writableMetadata = checkNotNull(writableMetadata); + this.rowType = checkNotNull(rowType); + } + + @Override + public void open( + SerializationSchema.InitializationContext initializationContext, + PulsarSinkContext sinkContext, + SinkConfiguration sinkConfiguration) + throws Exception { + valueSerialization.open(initializationContext); + } + + @Override + public PulsarMessage serialize(RowData consumedRow, PulsarSinkContext sinkContext) { + + PulsarMessageBuilder messageBuilder = new PulsarMessageBuilder<>(); + + final RowKind kind = consumedRow.getRowKind(); + final RowData valueRow = createProjectedRow(consumedRow, kind, valueFieldGetters); + + writableMetadata.applyWritableMetadataInMessage(consumedRow, messageBuilder); + + if (keySerialization != null) { + final RowData keyRow = createProjectedRow(consumedRow, RowKind.INSERT, keyFieldGetters); + messageBuilder.keyBytes(keySerialization.serialize(keyRow)); + } + + byte[] serializedData = valueSerialization.serialize(valueRow); + + // TODO, getting the schema everytime send message introduce an overhead, + // create a transient Schema field and set the Schema value in open(). + messageBuilder.value(getPulsarSchemaFromSerialization(), serializedData); + return messageBuilder.build(); + } + + private static RowData createProjectedRow( + RowData consumedRow, RowKind kind, RowData.FieldGetter[] fieldGetters) { + final int arity = fieldGetters.length; + final GenericRowData genericRowData = new GenericRowData(kind, arity); + for (int fieldPos = 0; fieldPos < arity; fieldPos++) { + genericRowData.setField(fieldPos, fieldGetters[fieldPos].getFieldOrNull(consumedRow)); + } + return genericRowData; + } + + /** A class to read fields from Flink row and map to a Pulsar metadata. */ + public interface MetadataConverter extends Serializable { + Object read(RowData consumedRow, int pos); + } + + private Schema getPulsarSchemaFromSerialization() { + if (valueSerialization instanceof AvroRowDataSerializationSchema) { + org.apache.avro.Schema avroSchema = AvroSchemaConverter.convertToSchema(rowType); + return Schema.NATIVE_AVRO(avroSchema); + } else if (valueSerialization instanceof JsonRowDataSerializationSchema) { + return new BytesSchema( + new PulsarSchema(Schema.JSON(SchemaDefinition.builder().build()))); + } else if (valueSerialization instanceof RawFormatSerializationSchema) { + return mapFromFlinkTypeToPulsarSchema(rowType); + } else { + return Schema.BYTES; + } + } + + private Schema mapFromFlinkTypeToPulsarSchema( + LogicalType type) { + switch (type.getTypeRoot()) { + case CHAR: + case VARCHAR: + return Schema.STRING; + case VARBINARY: + case BINARY: + case RAW: + return Schema.BYTES; + case BOOLEAN: + return Schema.BOOL; + case TINYINT: + return Schema.INT8; + case SMALLINT: + return Schema.INT16; + case INTEGER: + return Schema.INT32; + case BIGINT: + return Schema.INT64; + case FLOAT: + return Schema.FLOAT; + case DOUBLE: + return Schema.DOUBLE; + default: + throw new UnsupportedOperationException( + String.format("cannot map %s type to a pulsar primitive schema ", type)); + } + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchemaFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchemaFactory.java new file mode 100644 index 0000000000000..a2db75424020d --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchemaFactory.java @@ -0,0 +1,156 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.sink; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema; +import org.apache.flink.table.connector.Projection; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.DataTypeUtils; + +import javax.annotation.Nullable; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Contains needed field mapping and encoding format information to construct a {@link + * org.apache.flink.connector.pulsar.table.sink.PulsarTableSerializationSchema} instance. + */ +public class PulsarTableSerializationSchemaFactory { + + private final DataType physicalDataType; + + @Nullable private final EncodingFormat> keyEncodingFormat; + + private final int[] keyProjection; + + private final EncodingFormat> valueEncodingFormat; + + private final int[] valueProjection; + + /** Metadata that is appended at the end of a physical sink row. */ + private List writableMetadataKeys; + + public PulsarTableSerializationSchemaFactory( + DataType physicalDataType, + @Nullable EncodingFormat> keyEncodingFormat, + int[] keyProjection, + EncodingFormat> valueEncodingFormat, + int[] valueProjection) { + this.physicalDataType = checkNotNull(physicalDataType); + this.keyEncodingFormat = keyEncodingFormat; + this.keyProjection = checkNotNull(keyProjection); + this.valueEncodingFormat = checkNotNull(valueEncodingFormat); + this.valueProjection = checkNotNull(valueProjection); + this.writableMetadataKeys = Collections.emptyList(); + } + + public PulsarSerializationSchema createPulsarSerializationSchema( + DynamicTableSink.Context context) { + + final SerializationSchema keySerialization = + createSerialization(context, keyEncodingFormat, keyProjection, null); + + final SerializationSchema valueSerialization = + createSerialization(context, valueEncodingFormat, valueProjection, null); + final List physicalChildren = physicalDataType.getLogicalType().getChildren(); + + final RowData.FieldGetter[] keyFieldGetters = + getFieldGetters(physicalChildren, keyProjection); + final RowData.FieldGetter[] valueFieldGetters = + getFieldGetters(physicalChildren, valueProjection); + + final PulsarWritableMetadata writableMetadata = + new PulsarWritableMetadata(writableMetadataKeys, physicalChildren.size()); + + final RowType rowType = (RowType) physicalDataType.getLogicalType(); + return new PulsarTableSerializationSchema( + keySerialization, + keyFieldGetters, + valueSerialization, + valueFieldGetters, + writableMetadata, + rowType); + } + + private @Nullable SerializationSchema createSerialization( + DynamicTableSink.Context context, + @Nullable EncodingFormat> format, + int[] projection, + @Nullable String prefix) { + if (format == null) { + return null; + } + DataType physicalFormatDataType = Projection.of(projection).project(this.physicalDataType); + if (prefix != null) { + physicalFormatDataType = DataTypeUtils.stripRowPrefix(physicalFormatDataType, prefix); + } + return format.createRuntimeEncoder(context, physicalFormatDataType); + } + + private RowData.FieldGetter[] getFieldGetters( + List physicalChildren, int[] projection) { + return Arrays.stream(projection) + .mapToObj( + targetField -> + RowData.createFieldGetter( + physicalChildren.get(targetField), targetField)) + .toArray(RowData.FieldGetter[]::new); + } + + public void setWritableMetadataKeys(List writableMetadataKeys) { + this.writableMetadataKeys = writableMetadataKeys; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PulsarTableSerializationSchemaFactory that = (PulsarTableSerializationSchemaFactory) o; + return Objects.equals(physicalDataType, that.physicalDataType) + && Objects.equals(keyEncodingFormat, that.keyEncodingFormat) + && Arrays.equals(keyProjection, that.keyProjection) + && Objects.equals(valueEncodingFormat, that.valueEncodingFormat) + && Arrays.equals(valueProjection, that.valueProjection) + && Objects.equals(writableMetadataKeys, that.writableMetadataKeys); + } + + @Override + public int hashCode() { + int result = + Objects.hash( + physicalDataType, + keyEncodingFormat, + valueEncodingFormat, + writableMetadataKeys); + result = 31 * result + Arrays.hashCode(keyProjection); + result = 31 * result + Arrays.hashCode(valueProjection); + return result; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSink.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSink.java new file mode 100644 index 0000000000000..b290967873e5f --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSink.java @@ -0,0 +1,171 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.sink; + +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.pulsar.sink.PulsarSink; +import org.apache.flink.connector.pulsar.sink.PulsarSinkBuilder; +import org.apache.flink.connector.pulsar.sink.writer.delayer.FixedMessageDelayer; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; +import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkV2Provider; +import org.apache.flink.table.connector.sink.abilities.SupportsWritingMetadata; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; + +import javax.annotation.Nullable; + +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.stream.Stream; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Pulsar SQL Connector sink. It supports {@link SupportsWritingMetadata}. */ +public class PulsarTableSink implements DynamicTableSink, SupportsWritingMetadata { + + private final PulsarTableSerializationSchemaFactory serializationSchemaFactory; + + private final ChangelogMode changelogMode; + + private final List topics; + + private final Properties properties; + + private final DeliveryGuarantee deliveryGuarantee; + + @Nullable private final TopicRouter topicRouter; + + private final TopicRoutingMode topicRoutingMode; + + private final long messageDelayMillis; + + public PulsarTableSink( + PulsarTableSerializationSchemaFactory serializationSchemaFactory, + ChangelogMode changelogMode, + List topics, + Properties properties, + DeliveryGuarantee deliveryGuarantee, + @Nullable TopicRouter topicRouter, + TopicRoutingMode topicRoutingMode, + long messageDelayMillis) { + this.serializationSchemaFactory = checkNotNull(serializationSchemaFactory); + this.changelogMode = checkNotNull(changelogMode); + this.topics = checkNotNull(topics); + // Mutable attributes + this.properties = checkNotNull(properties); + this.deliveryGuarantee = checkNotNull(deliveryGuarantee); + this.topicRouter = topicRouter; + this.topicRoutingMode = checkNotNull(topicRoutingMode); + this.messageDelayMillis = messageDelayMillis; + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { + return this.changelogMode; + } + + @Override + public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { + + final PulsarSerializationSchema pulsarSerializationSchema = + serializationSchemaFactory.createPulsarSerializationSchema(context); + + final PulsarSinkBuilder pulsarSinkBuilder = + PulsarSink.builder() + .setSerializationSchema(pulsarSerializationSchema) + .setProperties(properties) + .setDeliveryGuarantee(deliveryGuarantee) + .setTopics(topics) + .setTopicRoutingMode(topicRoutingMode) + .delaySendingMessage(new FixedMessageDelayer<>(messageDelayMillis)); + + if (topicRouter != null) { + pulsarSinkBuilder.setTopicRouter(topicRouter); + } + return SinkV2Provider.of(pulsarSinkBuilder.build()); + } + + @Override + public String asSummaryString() { + return "Pulsar dynamic table sink"; + } + + @Override + public Map listWritableMetadata() { + final Map metadataMap = new LinkedHashMap<>(); + Stream.of(PulsarWritableMetadata.WritableMetadata.values()) + .forEachOrdered(m -> metadataMap.put(m.key, m.dataType)); + return metadataMap; + } + + @Override + public void applyWritableMetadata(List metadataKeys, DataType consumedDataType) { + this.serializationSchemaFactory.setWritableMetadataKeys(metadataKeys); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PulsarTableSink that = (PulsarTableSink) o; + return Objects.equals(serializationSchemaFactory, that.serializationSchemaFactory) + && Objects.equals(changelogMode, that.changelogMode) + && Objects.equals(topics, that.topics) + && Objects.equals(properties, that.properties) + && deliveryGuarantee == that.deliveryGuarantee + && Objects.equals(topicRouter, that.topicRouter) + && topicRoutingMode == that.topicRoutingMode + && messageDelayMillis == that.messageDelayMillis; + } + + @Override + public int hashCode() { + return Objects.hash( + serializationSchemaFactory, + changelogMode, + topics, + properties, + deliveryGuarantee, + topicRouter, + topicRoutingMode, + messageDelayMillis); + } + + @Override + public DynamicTableSink copy() { + final PulsarTableSink copy = + new PulsarTableSink( + serializationSchemaFactory, + changelogMode, + topics, + properties, + deliveryGuarantee, + topicRouter, + topicRoutingMode, + messageDelayMillis); + return copy; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarWritableMetadata.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarWritableMetadata.java new file mode 100644 index 0000000000000..8f25db73ec442 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarWritableMetadata.java @@ -0,0 +1,140 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.sink; + +import org.apache.flink.connector.pulsar.sink.writer.message.PulsarMessageBuilder; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; + +import org.bouncycastle.util.Arrays; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.stream.Stream; + +/** A class used to manage WritableMetadata for Pulsar SQL sink connector. */ +public class PulsarWritableMetadata implements Serializable { + + private static final long serialVersionUID = 8117156158379846715L; + + private final List writableMetadataKeys; + + private final int physicalChildrenSize; + + /** + * Contains the position for each value of {@link WritableMetadata} in the consumed row or -1 if + * this metadata key is not used. + */ + private int[] metadataPositions; + + public PulsarWritableMetadata(List writableMetadataKeys, int physicalChildrenSize) { + this.writableMetadataKeys = writableMetadataKeys; + this.physicalChildrenSize = physicalChildrenSize; + this.metadataPositions = getMetadataPositions(); + } + + public void applyWritableMetadataInMessage( + RowData consumedRow, PulsarMessageBuilder messageBuilder) { + Map properties = readMetadata(consumedRow, WritableMetadata.PROPERTIES); + if (properties != null) { + messageBuilder.properties(properties); + } + final Long eventTime = readMetadata(consumedRow, WritableMetadata.EVENT_TIME); + if (eventTime != null && eventTime >= 0) { + messageBuilder.eventTime(eventTime); + } + } + + @SuppressWarnings("unchecked") + private T readMetadata(RowData consumedRow, WritableMetadata metadata) { + if (Arrays.isNullOrEmpty(metadataPositions)) { + return null; + } + final int pos = metadataPositions[metadata.ordinal()]; + if (pos < 0) { + return null; + } + return (T) metadata.converter.read(consumedRow, pos); + } + + private int[] getMetadataPositions() { + return Stream.of(PulsarWritableMetadata.WritableMetadata.values()) + .mapToInt( + m -> { + final int pos = writableMetadataKeys.indexOf(m.key); + if (pos < 0) { + return -1; + } + return physicalChildrenSize + pos; + }) + .toArray(); + } + + /** A list of writable metadata used by Pulsar SQL sink connector. */ + public enum WritableMetadata { + PROPERTIES( + "properties", + // key and value of the map are nullable to make handling easier in queries + DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.STRING().nullable()) + .nullable(), + (row, pos) -> { + if (row.isNullAt(pos)) { + return null; + } + final MapData map = row.getMap(pos); + final ArrayData keyArray = map.keyArray(); + final ArrayData valueArray = map.valueArray(); + + final Properties properties = new Properties(); + for (int i = 0; i < keyArray.size(); i++) { + if (!keyArray.isNullAt(i) && !valueArray.isNullAt(i)) { + final String key = keyArray.getString(i).toString(); + final String value = valueArray.getString(i).toString(); + properties.put(key, value); + } + } + return properties; + }), + + EVENT_TIME( + "event_time", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), + (row, pos) -> { + if (row.isNullAt(pos)) { + return null; + } + return row.getTimestamp(pos, 3).getMillisecond(); + }); + public final String key; + + public final DataType dataType; + + public final PulsarTableSerializationSchema.MetadataConverter converter; + + WritableMetadata( + String key, + DataType dataType, + PulsarTableSerializationSchema.MetadataConverter converter) { + this.key = key; + this.dataType = dataType; + this.converter = converter; + } + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarReadableMetadata.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarReadableMetadata.java new file mode 100644 index 0000000000000..e0a7f9dc77557 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarReadableMetadata.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.source; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; + +import org.apache.pulsar.client.api.Message; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Class for reading metadata fields from a Pulsar message and put in corresponding Flink row + * fields. + * + *

Contains list of readable metadata and provide util methods for metadata manipulation. + */ +public class PulsarReadableMetadata implements Serializable { + + private static final long serialVersionUID = -4409932324481235973L; + + private final List connectorMetadataKeys; + + private final List metadataConverters; + + public PulsarReadableMetadata(List connectorMetadataKeys) { + this.connectorMetadataKeys = connectorMetadataKeys; + this.metadataConverters = initializeMetadataConverters(); + } + + private List initializeMetadataConverters() { + return connectorMetadataKeys.stream() + .map( + k -> + Stream.of(ReadableMetadata.values()) + .filter(rm -> rm.key.equals(k)) + .findFirst() + .orElseThrow(IllegalStateException::new)) + .map(m -> m.converter) + .collect(Collectors.toList()); + } + + public void appendProducedRowWithMetadata( + GenericRowData producedRowData, int physicalArity, Message message) { + for (int metadataPos = 0; metadataPos < metadataConverters.size(); metadataPos++) { + producedRowData.setField( + physicalArity + metadataPos, metadataConverters.get(metadataPos).read(message)); + } + } + + public int getConnectorMetadataArity() { + return metadataConverters.size(); + } + + // -------------------------------------------------------------------------------------------- + // Metadata handling + // -------------------------------------------------------------------------------------------- + interface MetadataConverter extends Serializable { + Object read(Message message); + } + + /** Lists the metadata that is readable from a Pulsar message. Used in SQL source connector. */ + public enum ReadableMetadata { + TOPIC( + "topic", + DataTypes.STRING().notNull(), + message -> StringData.fromString(message.getTopicName())), + + MESSAGE_SIZE("message_size", DataTypes.INT().notNull(), message -> message.size()), + + PRODUCER_NAME( + "producer_name", + DataTypes.STRING().notNull(), + message -> StringData.fromString(message.getProducerName())), + + MESSAGE_ID( + "message_id", + DataTypes.BYTES().notNull(), + message -> message.getMessageId().toByteArray()), + + SEQUENCE_ID("sequenceId", DataTypes.BIGINT().notNull(), Message::getSequenceId), + + PUBLISH_TIME( + "publish_time", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).notNull(), + message -> TimestampData.fromEpochMillis(message.getPublishTime())), + + EVENT_TIME( + "event_time", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).notNull(), + message -> TimestampData.fromEpochMillis(message.getEventTime())), + + PROPERTIES( + "properties", + // key and value of the map are nullable to make handling easier in queries + DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.STRING().nullable()) + .notNull(), + message -> { + final Map map = new HashMap<>(); + for (Map.Entry e : message.getProperties().entrySet()) { + map.put( + StringData.fromString(e.getKey()), + StringData.fromString(e.getValue())); + } + return new GenericMapData(map); + }); + + public final String key; + + public final DataType dataType; + + public final MetadataConverter converter; + + ReadableMetadata(String key, DataType dataType, MetadataConverter converter) { + this.key = key; + this.dataType = dataType; + this.converter = converter; + } + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarRowDataConverter.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarRowDataConverter.java new file mode 100644 index 0000000000000..5f8063669d6e7 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarRowDataConverter.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.source; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.DeserializationException; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.Collector; + +import org.apache.pulsar.client.api.Message; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.List; + +/** + * Contains the projection information needed to map a Pulsar message to proper key fields, value + * fields and metadata fields. + */ +public class PulsarRowDataConverter implements Serializable { + private static final long serialVersionUID = -3399264407634977459L; + + private final int physicalArity; + + private final int[] keyProjection; + + private final int[] valueProjection; + + private final PulsarReadableMetadata readableMetadata; + + public PulsarRowDataConverter( + int physicalArity, + int[] keyProjection, + int[] valueProjection, + PulsarReadableMetadata readableMetadata) { + this.physicalArity = physicalArity; + this.keyProjection = keyProjection; + this.valueProjection = valueProjection; + this.readableMetadata = readableMetadata; + } + + public void projectToProducedRowAndCollect( + Message message, + List keyRowDataList, + List valueRowDataList, + Collector collector) { + // no key defined + if (hasNoKeyProjection()) { + valueRowDataList.forEach( + valueRow -> emitRow(null, (GenericRowData) valueRow, collector, message)); + } else { + // otherwise emit a value for each key + valueRowDataList.forEach( + valueRow -> + keyRowDataList.forEach( + keyRow -> + emitRow( + (GenericRowData) keyRow, + (GenericRowData) valueRow, + collector, + message))); + } + } + + private void emitRow( + @Nullable GenericRowData physicalKeyRow, + @Nullable GenericRowData physicalValueRow, + Collector collector, + Message message) { + + final RowKind rowKind; + if (physicalValueRow == null) { + throw new DeserializationException( + "Invalid null value received in non-upsert mode. Could not to set row kind for output record." + + "upsert mode is not supported yet."); + } else { + rowKind = physicalValueRow.getRowKind(); + } + + final GenericRowData producedRow = + new GenericRowData( + rowKind, physicalArity + readableMetadata.getConnectorMetadataArity()); + + for (int valuePos = 0; valuePos < valueProjection.length; valuePos++) { + producedRow.setField(valueProjection[valuePos], physicalValueRow.getField(valuePos)); + } + + for (int keyPos = 0; keyPos < keyProjection.length; keyPos++) { + assert physicalKeyRow != null; + producedRow.setField(keyProjection[keyPos], physicalKeyRow.getField(keyPos)); + } + + readableMetadata.appendProducedRowWithMetadata(producedRow, physicalArity, message); + collector.collect(producedRow); + } + + private boolean hasNoKeyProjection() { + return keyProjection.length == 0; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchema.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchema.java new file mode 100644 index 0000000000000..ae19e222daa5b --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchema.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.source; + +import org.apache.flink.api.common.functions.util.ListCollector; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.Collector; + +import org.apache.pulsar.client.api.Message; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A specific {@link PulsarDeserializationSchema} for {@link PulsarTableSource}. + * + *

Both Flink's key decoding format and value decoding format are wrapped in this class. It is + * responsible for getting metadata fields from a physical pulsar message body, and the final + * projection mapping from Pulsar message fields to Flink row. + * + *

After retrieving key and value bytes and convert them into a list of {@link RowData}, it then + * delegates metadata appending, key and value {@link RowData} combining to a {@link + * PulsarRowDataConverter} instance. + */ +public class PulsarTableDeserializationSchema implements PulsarDeserializationSchema { + private static final long serialVersionUID = -3298784447432136216L; + + private final TypeInformation producedTypeInfo; + + @Nullable private final DeserializationSchema keyDeserialization; + + private final DeserializationSchema valueDeserialization; + + private final PulsarRowDataConverter rowDataConverter; + + public PulsarTableDeserializationSchema( + @Nullable DeserializationSchema keyDeserialization, + DeserializationSchema valueDeserialization, + TypeInformation producedTypeInfo, + PulsarRowDataConverter rowDataConverter) { + this.keyDeserialization = keyDeserialization; + this.valueDeserialization = checkNotNull(valueDeserialization); + this.rowDataConverter = checkNotNull(rowDataConverter); + this.producedTypeInfo = checkNotNull(producedTypeInfo); + } + + @Override + public void open( + DeserializationSchema.InitializationContext context, SourceConfiguration configuration) + throws Exception { + if (keyDeserialization != null) { + keyDeserialization.open(context); + } + valueDeserialization.open(context); + } + + @Override + public void deserialize(Message message, Collector collector) + throws IOException { + // Get the value row data + List valueRowData = new ArrayList<>(); + valueDeserialization.deserialize(message.getData(), new ListCollector<>(valueRowData)); + + // Get the key row data + List keyRowData = new ArrayList<>(); + if (keyDeserialization != null) { + keyDeserialization.deserialize(message.getKeyBytes(), new ListCollector<>(keyRowData)); + } + + rowDataConverter.projectToProducedRowAndCollect( + message, keyRowData, valueRowData, collector); + } + + @Override + public TypeInformation getProducedType() { + return producedTypeInfo; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchemaFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchemaFactory.java new file mode 100644 index 0000000000000..b583cf667df9a --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchemaFactory.java @@ -0,0 +1,204 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.source; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; +import org.apache.flink.table.connector.Projection; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.utils.DataTypeUtils; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.stream.IntStream; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Contains key, value projection and format information, and use such information to create a + * {@link org.apache.flink.connector.pulsar.table.source.PulsarTableDeserializationSchema} instance + * used by runtime {@link org.apache.flink.connector.pulsar.source.PulsarSource} instance. + * + *

A Flink row fields has a strict order: Physical Fields (Key + value) + Format Metadata Fields + * Connector Metadata Fields. Physical Fields are fields come directly from Pulsar message body; + * Format Metadata Fields are from the extra information from the decoding format. Connector + * metadata fields are the ones most Pulsar messages have, such as publish time, message size and + * producer name. + * + *

In general, Physical fields + Format Metadata fields are contained in the RowData decoded + * using valueDecodingFormat. Only Connector Metadata fields needs to be appended to the decoded + * RowData. The tricky part is to put format metadata and connector metadata in the right location. + * This requires an explicit adjustment process. + * + *

For example, suppose Physical Fields (Key + value) + Format Metadata Fields + Connector + * Metadata Fields. has arity of 11, key projection is [0, 6], and physical value projection is [1, + * 2, 3, 4, 5], Then after the adjustment, key projection should be [0, 6], physical value + * projection should be [1, 2, 3, 4, 5] and format metadata projection should be [7], connector + * metadata projection should be [8, 9, 10]. + */ +public class PulsarTableDeserializationSchemaFactory implements Serializable { + + private static final long serialVersionUID = 6091562041940740434L; + + private final DataType physicalDataType; + + @Nullable private final DecodingFormat> keyDecodingFormat; + + private final int[] keyProjection; + + private final DecodingFormat> valueDecodingFormat; + + private final int[] valueProjection; + + // -------------------------------------------------------------------------------------------- + // Mutable attributes. Will be updated after the applyReadableMetadata() + // -------------------------------------------------------------------------------------------- + private DataType producedDataType; + + private List connectorMetadataKeys; + + public PulsarTableDeserializationSchemaFactory( + DataType physicalDataType, + @Nullable DecodingFormat> keyDecodingFormat, + int[] keyProjection, + DecodingFormat> valueDecodingFormat, + int[] valueProjection) { + this.physicalDataType = + checkNotNull(physicalDataType, "field physicalDataType must not be null."); + this.keyDecodingFormat = keyDecodingFormat; + this.keyProjection = checkNotNull(keyProjection); + this.valueDecodingFormat = + checkNotNull(valueDecodingFormat, "field valueDecodingFormat must not be null."); + this.valueProjection = + checkNotNull(valueProjection, "field valueProjection must not be null."); + + this.producedDataType = physicalDataType; + this.connectorMetadataKeys = Collections.emptyList(); + } + + private @Nullable DeserializationSchema createDeserialization( + DynamicTableSource.Context context, + @Nullable DecodingFormat> format, + int[] projection, + @Nullable String prefix) { + if (format == null) { + return null; + } + + DataType physicalFormatDataType = Projection.of(projection).project(this.physicalDataType); + if (prefix != null) { + physicalFormatDataType = DataTypeUtils.stripRowPrefix(physicalFormatDataType, prefix); + } + return format.createRuntimeDecoder(context, physicalFormatDataType); + } + + public PulsarDeserializationSchema createPulsarDeserialization( + ScanTableSource.ScanContext context) { + final DeserializationSchema keyDeserialization = + createDeserialization(context, keyDecodingFormat, keyProjection, ""); + final DeserializationSchema valueDeserialization = + createDeserialization(context, valueDecodingFormat, valueProjection, ""); + + final TypeInformation producedTypeInfo = + context.createTypeInformation(producedDataType); + + final PulsarReadableMetadata readableMetadata = + new PulsarReadableMetadata(connectorMetadataKeys); + + // Get Physical Fields (key + value) + Format Metadata arity + final int physicalPlusFormatMetadataArity = + DataType.getFieldDataTypes(producedDataType).size() + - readableMetadata.getConnectorMetadataArity(); + final int[] physicalValuePlusFormatMetadataProjection = + adjustValueProjectionByAppendConnectorMetadata(physicalPlusFormatMetadataArity); + + final PulsarRowDataConverter rowDataConverter = + new PulsarRowDataConverter( + physicalPlusFormatMetadataArity, + keyProjection, + physicalValuePlusFormatMetadataProjection, + readableMetadata); + + return new PulsarTableDeserializationSchema( + keyDeserialization, valueDeserialization, producedTypeInfo, rowDataConverter); + } + + public void setProducedDataType(DataType producedDataType) { + this.producedDataType = producedDataType; + } + + public void setConnectorMetadataKeys(List metadataKeys) { + this.connectorMetadataKeys = metadataKeys; + } + + private int[] adjustValueProjectionByAppendConnectorMetadata( + int physicalValuePlusFormatMetadataArity) { + // Concat the Physical Fields (value only) with Format metadata projection. + final int[] physicalValuePlusFormatMetadataProjection = + IntStream.concat( + IntStream.of(valueProjection), + IntStream.range( + keyProjection.length + valueProjection.length, + physicalValuePlusFormatMetadataArity)) + .toArray(); + return physicalValuePlusFormatMetadataProjection; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PulsarTableDeserializationSchemaFactory that = (PulsarTableDeserializationSchemaFactory) o; + return Objects.equals(physicalDataType, that.physicalDataType) + && Objects.equals(keyDecodingFormat, that.keyDecodingFormat) + && Arrays.equals(keyProjection, that.keyProjection) + && Objects.equals(valueDecodingFormat, that.valueDecodingFormat) + && Arrays.equals(valueProjection, that.valueProjection) + && Objects.equals(producedDataType, that.producedDataType) + && Objects.equals(connectorMetadataKeys, that.connectorMetadataKeys); + } + + @Override + public int hashCode() { + int result = + Objects.hash( + physicalDataType, + keyDecodingFormat, + valueDecodingFormat, + producedDataType, + connectorMetadataKeys); + result = 31 * result + Arrays.hashCode(keyProjection); + result = 31 * result + Arrays.hashCode(valueProjection); + return result; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableSource.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableSource.java new file mode 100644 index 0000000000000..a677256471f2f --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableSource.java @@ -0,0 +1,222 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.source; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.connector.pulsar.source.PulsarSource; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.connector.source.SourceProvider; +import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; + +import org.apache.pulsar.client.api.SubscriptionType; + +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link ScanTableSource} implementation for Pulsar SQL Connector. It uses a {@link + * SourceProvider} so it doesn't need to support {@link + * org.apache.flink.table.connector.source.abilities.SupportsWatermarkPushDown} interface. + * + *

{@link PulsarTableSource} + */ +public class PulsarTableSource implements ScanTableSource, SupportsReadingMetadata { + // -------------------------------------------------------------------------------------------- + // Format attributes + // -------------------------------------------------------------------------------------------- + + private static final String FORMAT_METADATA_PREFIX = "value."; + + private final PulsarTableDeserializationSchemaFactory deserializationSchemaFactory; + + /** + * Usually it is the same as the valueDecodingFormat, but use a different naming to show that it + * is used to list all the format metadata keys. + */ + private final DecodingFormat> decodingFormatForReadingMetadata; + + // -------------------------------------------------------------------------------------------- + // PulsarSource needed attributes + // -------------------------------------------------------------------------------------------- + + private final List topics; + + private final Properties properties; + + private final StartCursor startCursor; + + private final StopCursor stopCursor; + + private final SubscriptionType subscriptionType; + + public PulsarTableSource( + PulsarTableDeserializationSchemaFactory deserializationSchemaFactory, + DecodingFormat> decodingFormatForReadingMetadata, + List topics, + Properties properties, + StartCursor startCursor, + StopCursor stopCursor, + SubscriptionType subscriptionType) { + // Format attributes + this.deserializationSchemaFactory = checkNotNull(deserializationSchemaFactory); + this.decodingFormatForReadingMetadata = checkNotNull(decodingFormatForReadingMetadata); + // DataStream connector attributes + this.topics = topics; + this.properties = checkNotNull(properties); + this.startCursor = checkNotNull(startCursor); + this.stopCursor = checkNotNull(stopCursor); + this.subscriptionType = checkNotNull(subscriptionType); + } + + @Override + public ChangelogMode getChangelogMode() { + return decodingFormatForReadingMetadata.getChangelogMode(); + } + + @Override + public ScanRuntimeProvider getScanRuntimeProvider(ScanContext context) { + PulsarDeserializationSchema deserializationSchema = + deserializationSchemaFactory.createPulsarDeserialization(context); + PulsarSource source = + PulsarSource.builder() + .setTopics(topics) + .setStartCursor(startCursor) + .setUnboundedStopCursor(stopCursor) + .setDeserializationSchema(deserializationSchema) + .setSubscriptionType(subscriptionType) + .setProperties(properties) + .build(); + return SourceProvider.of(source); + } + + /** + * According to convention, the order of the final row must be PHYSICAL + FORMAT METADATA + + * CONNECTOR METADATA where the format metadata has the highest precedence. + * + * @return + */ + @Override + public Map listReadableMetadata() { + final Map allMetadataMap = new LinkedHashMap<>(); + + // add value format metadata with prefix + decodingFormatForReadingMetadata + .listReadableMetadata() + .forEach((key, value) -> allMetadataMap.put(FORMAT_METADATA_PREFIX + key, value)); + // add connector metadata + Stream.of(PulsarReadableMetadata.ReadableMetadata.values()) + .forEachOrdered(m -> allMetadataMap.putIfAbsent(m.key, m.dataType)); + + return allMetadataMap; + } + + @Override + public void applyReadableMetadata(List allMetadataKeys, DataType producedDataType) { + // separate connector and format metadata + final List formatMetadataKeys = + allMetadataKeys.stream() + .filter(k -> k.startsWith(FORMAT_METADATA_PREFIX)) + .collect(Collectors.toList()); + + final List connectorMetadataKeys = new ArrayList<>(allMetadataKeys); + connectorMetadataKeys.removeAll(formatMetadataKeys); + + // push down format metadata + final Map formatMetadata = + decodingFormatForReadingMetadata.listReadableMetadata(); + if (formatMetadata.size() > 0) { + final List requestedFormatMetadataKeys = + formatMetadataKeys.stream() + .map(k -> k.substring(FORMAT_METADATA_PREFIX.length())) + .collect(Collectors.toList()); + decodingFormatForReadingMetadata.applyReadableMetadata(requestedFormatMetadataKeys); + } + + // update the factory attributes. + deserializationSchemaFactory.setConnectorMetadataKeys(connectorMetadataKeys); + deserializationSchemaFactory.setProducedDataType(producedDataType); + } + + @Override + public String asSummaryString() { + return "Pulsar table source"; + } + + @Override + public DynamicTableSource copy() { + final PulsarTableSource copy = + new PulsarTableSource( + deserializationSchemaFactory, + decodingFormatForReadingMetadata, + topics, + properties, + startCursor, + stopCursor, + subscriptionType); + return copy; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + PulsarTableSource that = (PulsarTableSource) o; + return Objects.equals(deserializationSchemaFactory, that.deserializationSchemaFactory) + && Objects.equals( + decodingFormatForReadingMetadata, that.decodingFormatForReadingMetadata) + && Objects.equals(topics, that.topics) + && Objects.equals(properties, that.properties) + && Objects.equals(startCursor, that.startCursor) + && Objects.equals(stopCursor, that.stopCursor) + && subscriptionType == that.subscriptionType; + } + + @Override + public int hashCode() { + return Objects.hash( + deserializationSchemaFactory, + decodingFormatForReadingMetadata, + topics, + properties, + startCursor, + stopCursor, + subscriptionType); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connectors/flink-connector-pulsar/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 0000000000000..c2e04bc746218 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -0,0 +1,18 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.flink.connector.pulsar.table.PulsarTableFactory +org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogFactory + diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java index 7565193ca9a42..b99a631d92548 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java @@ -34,7 +34,6 @@ ImportOptions.ExcludeShadedImportOption.class }) public class TestCodeArchitectureTest { - @ArchTest public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class); } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/PulsarSchemaTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/PulsarSchemaTest.java index 7011e169656d3..81074c993708f 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/PulsarSchemaTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/PulsarSchemaTest.java @@ -36,6 +36,8 @@ import org.apache.pulsar.common.schema.SchemaType; import org.junit.jupiter.api.Test; +import java.io.Serializable; + import static org.junit.jupiter.api.Assertions.assertAll; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -117,9 +119,35 @@ void pulsarSchemaSerialization() throws Exception { assertPulsarSchemaIsSerializable(new PulsarSchema<>(KV, Foo.class, FA.class)); } + @Test + void largeAvroSchemaSerialization() throws Exception { + Schema largeMessageSchema = Schema.AVRO(LargeMessage.class); + assertPulsarSchemaIsSerializable( + new PulsarSchema<>(largeMessageSchema, LargeMessage.class)); + } + private void assertPulsarSchemaIsSerializable(PulsarSchema schema) throws Exception { PulsarSchema clonedSchema = InstantiationUtil.clone(schema); assertEquals(clonedSchema.getSchemaInfo(), schema.getSchemaInfo()); assertEquals(clonedSchema.getRecordClass(), schema.getRecordClass()); } + + /** A POJO Class which would generate a large schema by Avro. */ + public static class LargeMessage implements Serializable { + private static final long serialVersionUID = 5364494369740402518L; + + public String + aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa; + public String + bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb; + public String + cccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccc; + public String + dddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddd; + public String + eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee; + // the problem begins + public String + ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff; + } } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactoryTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactoryTest.java index e4b2dc1b9e9cf..d67b64ceac9e7 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactoryTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactoryTest.java @@ -20,9 +20,9 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.connector.pulsar.common.schema.PulsarSchema; -import org.apache.flink.connector.pulsar.common.schema.PulsarSchemaTypeInformation; import org.apache.flink.connector.pulsar.testutils.SampleData.FL; import org.apache.flink.connector.pulsar.testutils.SampleData.Foo; +import org.apache.flink.formats.avro.typeutils.AvroTypeInfo; import org.apache.flink.util.InstantiationUtil; import org.apache.pulsar.client.api.Schema; @@ -62,7 +62,7 @@ void createJSONTypeInformationFromSchemaInfo() { TypeInformation typeInfo = factory.createTypeInfo(pulsarSchema.getSchemaInfo()); assertThat(typeInfo) - .isInstanceOf(PulsarSchemaTypeInformation.class) + .isInstanceOf(AvroTypeInfo.class) .hasFieldOrPropertyWithValue("typeClass", FL.class); // TypeInformation serialization. diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/PulsarSinkITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/PulsarSinkITCase.java index 94e23d7b0977e..63113731c4bab 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/PulsarSinkITCase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/PulsarSinkITCase.java @@ -20,14 +20,26 @@ import org.apache.flink.api.common.serialization.SimpleStringSchema; import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.pulsar.testutils.PulsarTestContextFactory; +import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; import org.apache.flink.connector.pulsar.testutils.PulsarTestSuiteBase; import org.apache.flink.connector.pulsar.testutils.function.ControlSource; +import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntime; +import org.apache.flink.connector.pulsar.testutils.sink.PulsarSinkTestContext; +import org.apache.flink.connector.pulsar.testutils.sink.PulsarSinkTestSuiteBase; +import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment; +import org.apache.flink.connector.testframe.junit.annotations.TestContext; +import org.apache.flink.connector.testframe.junit.annotations.TestEnv; +import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem; +import org.apache.flink.connector.testframe.junit.annotations.TestSemantics; import org.apache.flink.runtime.minicluster.RpcServiceSharing; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.testutils.junit.SharedObjectsExtension; +import org.junit.jupiter.api.Nested; import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; @@ -43,52 +55,87 @@ /** Tests for using PulsarSink writing to a Pulsar cluster. */ class PulsarSinkITCase extends PulsarTestSuiteBase { - private static final int PARALLELISM = 1; - - @RegisterExtension - private static final MiniClusterExtension MINI_CLUSTER_RESOURCE = - new MiniClusterExtension( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(1) - .setNumberSlotsPerTaskManager(PARALLELISM) - .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .withHaLeadershipControl() - .build()); - - // Using this extension for creating shared reference which would be used in source function. - @RegisterExtension final SharedObjectsExtension sharedObjects = SharedObjectsExtension.create(); - - @ParameterizedTest - @EnumSource(DeliveryGuarantee.class) - void writeRecordsToPulsar(DeliveryGuarantee guarantee) throws Exception { - // A random topic with partition 1. - String topic = randomAlphabetic(8); - operator().createTopic(topic, 4); - int counts = ThreadLocalRandom.current().nextInt(100, 200); - - ControlSource source = - new ControlSource( - sharedObjects, operator(), topic, guarantee, counts, Duration.ofMinutes(5)); - PulsarSink sink = - PulsarSink.builder() - .setServiceUrl(operator().serviceUrl()) - .setAdminUrl(operator().adminUrl()) - .setDeliveryGuarantee(guarantee) - .setTopics(topic) - .setSerializationSchema(flinkSchema(new SimpleStringSchema())) - .build(); - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(PARALLELISM); - env.enableCheckpointing(100L); - env.addSource(source).sinkTo(sink); - env.execute(); - - List expectedRecords = source.getExpectedRecords(); - List consumedRecords = source.getConsumedRecords(); - - assertThat(consumedRecords) - .hasSameSizeAs(expectedRecords) - .containsExactlyInAnyOrderElementsOf(expectedRecords); + /** Integration test based on connector testing framework. */ + @Nested + class IntegrationTest extends PulsarSinkTestSuiteBase { + + @TestEnv MiniClusterTestEnvironment flink = new MiniClusterTestEnvironment(); + + @TestExternalSystem + PulsarTestEnvironment pulsar = new PulsarTestEnvironment(PulsarRuntime.mock()); + + @TestSemantics + CheckpointingMode[] semantics = + new CheckpointingMode[] { + CheckpointingMode.EXACTLY_ONCE, CheckpointingMode.AT_LEAST_ONCE + }; + + @TestContext + PulsarTestContextFactory sinkContext = + new PulsarTestContextFactory<>(pulsar, PulsarSinkTestContext::new); + } + + /** Tests for using PulsarSink writing to a Pulsar cluster. */ + @Nested + class DeliveryGuaranteeTest extends PulsarTestSuiteBase { + + private static final int PARALLELISM = 1; + + @RegisterExtension + private final MiniClusterExtension clusterExtension = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .withHaLeadershipControl() + .build()); + + // Using this extension for creating shared reference which would be used in source + // function. + @RegisterExtension + final SharedObjectsExtension sharedObjects = SharedObjectsExtension.create(); + + @ParameterizedTest + @EnumSource(DeliveryGuarantee.class) + void writeRecordsToPulsar(DeliveryGuarantee guarantee) throws Exception { + // A random topic with partition 4. + String topic = randomAlphabetic(8); + operator().createTopic(topic, 4); + int counts = ThreadLocalRandom.current().nextInt(100, 200); + + ControlSource source = + new ControlSource( + sharedObjects, + operator(), + topic, + guarantee, + counts, + Duration.ofMillis(50)); + PulsarSink sink = + PulsarSink.builder() + .setServiceUrl(operator().serviceUrl()) + .setAdminUrl(operator().adminUrl()) + .setDeliveryGuarantee(guarantee) + .setTopics(topic) + .setSerializationSchema(flinkSchema(new SimpleStringSchema())) + .build(); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + env.setParallelism(PARALLELISM); + if (guarantee != DeliveryGuarantee.NONE) { + env.enableCheckpointing(500L); + } + env.addSource(source).sinkTo(sink); + env.execute(); + + List expectedRecords = source.getExpectedRecords(); + List consumedRecords = source.getConsumedRecords(); + + assertThat(consumedRecords) + .hasSameSizeAs(expectedRecords) + .containsExactlyInAnyOrderElementsOf(expectedRecords); + } } } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriterTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriterTest.java index 942b75952b8b2..9fc272a7338c8 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriterTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriterTest.java @@ -31,7 +31,7 @@ import org.apache.flink.connector.pulsar.sink.writer.delayer.MessageDelayer; import org.apache.flink.connector.pulsar.sink.writer.router.RoundRobinTopicRouter; import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema; -import org.apache.flink.connector.pulsar.sink.writer.topic.TopicMetadataListener; +import org.apache.flink.connector.pulsar.sink.writer.topic.register.FixedTopicRegister; import org.apache.flink.connector.pulsar.testutils.PulsarTestSuiteBase; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.metrics.groups.OperatorIOMetricGroup; @@ -80,13 +80,14 @@ void writeMessageWithoutGuarantee(DeliveryGuarantee guarantee) throws Exception SinkConfiguration configuration = sinkConfiguration(guarantee); PulsarSerializationSchema schema = pulsarSchema(STRING); - TopicMetadataListener listener = new TopicMetadataListener(singletonList(topic)); + FixedTopicRegister listener = new FixedTopicRegister(singletonList(topic)); RoundRobinTopicRouter router = new RoundRobinTopicRouter<>(configuration); FixedMessageDelayer delayer = MessageDelayer.never(); MockInitContext initContext = new MockInitContext(); PulsarWriter writer = - new PulsarWriter<>(configuration, schema, listener, router, delayer, initContext); + new PulsarWriter<>( + configuration, schema, listener, router, delayer, null, initContext); writer.flush(false); writer.prepareCommit(); diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicProducerRegisterTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/ProducerRegisterTest.java similarity index 92% rename from flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicProducerRegisterTest.java rename to flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/ProducerRegisterTest.java index 2e36bfb0cf758..135ce8779b3f3 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicProducerRegisterTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/ProducerRegisterTest.java @@ -38,8 +38,8 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; -/** Unit tests for {@link TopicProducerRegister}. */ -class TopicProducerRegisterTest extends PulsarTestSuiteBase { +/** Unit tests for {@link ProducerRegister}. */ +class ProducerRegisterTest extends PulsarTestSuiteBase { @ParameterizedTest @EnumSource(DeliveryGuarantee.class) @@ -49,7 +49,7 @@ void createMessageBuilderForSendingMessage(DeliveryGuarantee deliveryGuarantee) operator().createTopic(topic, 8); SinkConfiguration configuration = sinkConfiguration(deliveryGuarantee); - TopicProducerRegister register = new TopicProducerRegister(configuration); + ProducerRegister register = new ProducerRegister(configuration, null); String message = randomAlphabetic(10); register.createMessageBuilder(topic, Schema.STRING).value(message).send(); @@ -76,7 +76,7 @@ void noneAndAtLeastOnceWouldNotCreateTransaction(DeliveryGuarantee deliveryGuara operator().createTopic(topic, 8); SinkConfiguration configuration = sinkConfiguration(deliveryGuarantee); - TopicProducerRegister register = new TopicProducerRegister(configuration); + ProducerRegister register = new ProducerRegister(configuration, null); String message = randomAlphabetic(10); register.createMessageBuilder(topic, Schema.STRING).value(message).sendAsync(); diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/CachedTopicMetadataProviderTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/CachedTopicMetadataProviderTest.java new file mode 100644 index 0000000000000..1bef64b8857af --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/CachedTopicMetadataProviderTest.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.sink.writer.topic.metadata; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata; +import org.apache.flink.connector.pulsar.testutils.PulsarTestSuiteBase; + +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; +import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_TOPIC_METADATA_REFRESH_INTERVAL; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; + +/** Unit tests for {@link CachedTopicMetadataProvider}. */ +class CachedTopicMetadataProviderTest extends PulsarTestSuiteBase { + + @Test + void queryTopicsWhichIsNotExisted() { + CachedTopicMetadataProvider provider = + new CachedTopicMetadataProvider( + operator().admin(), new SinkConfiguration(new Configuration())); + + String notExistedTopic = "not-existed-topic-" + randomAlphanumeric(8); + + assertFalse(operator().topicExists(notExistedTopic)); + assertThrows(PulsarAdminException.class, () -> provider.query(notExistedTopic)); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void queryTopicsWhichIsExisted(boolean enableCache) throws PulsarAdminException { + String topicName = "existed-topic-" + randomAlphanumeric(8); + operator().createTopic(topicName, 8); + + Configuration configuration = new Configuration(); + if (!enableCache) { + configuration.set(PULSAR_TOPIC_METADATA_REFRESH_INTERVAL, -1L); + } + + CachedTopicMetadataProvider provider = + new CachedTopicMetadataProvider( + operator().admin(), new SinkConfiguration(configuration)); + + TopicMetadata metadata1 = provider.query(topicName); + assertThat(metadata1).hasFieldOrPropertyWithValue("partitionSize", 8); + + // Increase topic partition, but the query result didn't get changed immediately with cache. + operator().increaseTopicPartitions(topicName, 16); + + TopicMetadata metadata2 = provider.query(topicName); + if (enableCache) { + assertThat(metadata2).hasFieldOrPropertyWithValue("partitionSize", 8); + } else { + assertThat(metadata2).hasFieldOrPropertyWithValue("partitionSize", 16); + } + } + + @Test + void queryTopicsWhichIsExistedWithoutCache() { + String topicName = "existed-topic-" + randomAlphanumeric(8); + operator().createTopic(topicName, 8); + + Configuration configuration = new Configuration(); + configuration.set(PULSAR_TOPIC_METADATA_REFRESH_INTERVAL, -1L); + CachedTopicMetadataProvider provider = + new CachedTopicMetadataProvider( + operator().admin(), new SinkConfiguration(configuration)); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/NotExistedTopicMetadataProviderTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/NotExistedTopicMetadataProviderTest.java new file mode 100644 index 0000000000000..770b141a49697 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/NotExistedTopicMetadataProviderTest.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.sink.writer.topic.metadata; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata; +import org.apache.flink.connector.pulsar.testutils.PulsarTestSuiteBase; + +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.junit.jupiter.api.Test; + +import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_SINK_DEFAULT_TOPIC_PARTITIONS; +import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_SINK_TOPIC_AUTO_CREATION; +import static org.assertj.core.api.Assertions.assertThat; + +/** Unit tests for {@link NotExistedTopicMetadataProvider}. */ +class NotExistedTopicMetadataProviderTest extends PulsarTestSuiteBase { + + @Test + void autoTopicCreationForNotExisted() throws PulsarAdminException { + // Create an existed topic + operator().createTopic("existed-topic", 10); + + // This provider will create a topic with 5 partitions. + NotExistedTopicMetadataProvider provider1 = + new NotExistedTopicMetadataProvider(operator().admin(), configuration(5)); + + TopicMetadata metadata1 = provider1.query("existed-topic"); + assertThat(metadata1).hasFieldOrPropertyWithValue("partitionSize", 10); + + TopicMetadata metadata2 = provider1.query("not-existed-topic-1"); + assertThat(metadata2).hasFieldOrPropertyWithValue("partitionSize", 5); + + // This provider will create a topic with 8 partitions. + NotExistedTopicMetadataProvider provider2 = + new NotExistedTopicMetadataProvider(operator().admin(), configuration(8)); + + TopicMetadata metadata3 = provider2.query("not-existed-topic-1"); + assertThat(metadata3).hasFieldOrPropertyWithValue("partitionSize", 5); + + TopicMetadata metadata4 = provider2.query("not-existed-topic-2"); + assertThat(metadata4).hasFieldOrPropertyWithValue("partitionSize", 8); + } + + private SinkConfiguration configuration(int partitions) { + Configuration configuration = new Configuration(); + configuration.set(PULSAR_SINK_TOPIC_AUTO_CREATION, true); + configuration.set(PULSAR_SINK_DEFAULT_TOPIC_PARTITIONS, partitions); + + return new SinkConfiguration(configuration); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/DynamicTopicRegisterTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/DynamicTopicRegisterTest.java new file mode 100644 index 0000000000000..c5290d6bf78a8 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/DynamicTopicRegisterTest.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.sink.writer.topic.register; + +import org.apache.flink.api.common.operators.ProcessingTimeService; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; +import org.apache.flink.connector.pulsar.sink.writer.topic.TopicExtractor; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; +import org.apache.flink.connector.pulsar.testutils.PulsarTestSuiteBase; + +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.List; + +import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; +import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_TOPIC_METADATA_REFRESH_INTERVAL; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; + +/** Unit tests for {@link DynamicTopicRegister}. */ +class DynamicTopicRegisterTest extends PulsarTestSuiteBase { + + private final MockTopicExtractor extractor = new MockTopicExtractor(); + + private static final class MockTopicExtractor implements TopicExtractor { + private static final long serialVersionUID = 2456172645787498006L; + + private TopicPartition partition; + + @Override + public TopicPartition extract(String s, TopicMetadataProvider provider) { + return partition; + } + + public void setPartition(TopicPartition partition) { + this.partition = partition; + } + } + + @Test + void partitionedTopicWouldBeReturnedDirectly() throws IOException { + DynamicTopicRegister register = topicRegister(50000); + TopicPartition partition = new TopicPartition("some", 1); + extractor.setPartition(partition); + List topics = register.topics(randomAlphabetic(10)); + + assertThat(topics) + .hasSize(1) + .allSatisfy(topic -> assertThat(topic).isEqualTo(partition.getFullTopicName())); + + register.close(); + } + + @Test + void rootTopicWillReturnAllThePartitions() throws IOException { + DynamicTopicRegister register = topicRegister(50000); + TopicPartition partition = new TopicPartition("root-topic" + randomAlphabetic(10)); + extractor.setPartition(partition); + operator().createTopic(partition.getFullTopicName(), 10); + List topics = register.topics(randomAlphabetic(10)); + + assertThat(topics) + .hasSize(10) + .allSatisfy(topic -> assertThat(topic).startsWith(partition.getTopic())); + + register.close(); + } + + private DynamicTopicRegister topicRegister(long interval) { + DynamicTopicRegister register = new DynamicTopicRegister<>(extractor); + register.open(sinkConfiguration(interval), mock(ProcessingTimeService.class)); + + return register; + } + + private SinkConfiguration sinkConfiguration(long interval) { + Configuration configuration = operator().config(); + configuration.set(PULSAR_TOPIC_METADATA_REFRESH_INTERVAL, interval); + + return new SinkConfiguration(configuration); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicMetadataListenerTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/FixedTopicRegisterTest.java similarity index 69% rename from flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicMetadataListenerTest.java rename to flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/FixedTopicRegisterTest.java index f65b238738567..b0ee0c5ddc7cc 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicMetadataListenerTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/FixedTopicRegisterTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.connector.pulsar.sink.writer.topic; +package org.apache.flink.connector.pulsar.sink.writer.topic.register; import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; @@ -25,33 +25,39 @@ import org.junit.jupiter.api.Test; +import java.io.IOException; import java.time.Duration; +import java.util.Collections; import java.util.List; import java.util.stream.IntStream; +import static java.util.Collections.emptyList; import static java.util.Collections.singletonList; import static java.util.stream.Collectors.toList; import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_TOPIC_METADATA_REFRESH_INTERVAL; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicName; import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; -/** Unit tests for {@link TopicMetadataListener}. */ -class TopicMetadataListenerTest extends PulsarTestSuiteBase { +/** Unit tests for {@link FixedTopicRegister}. */ +class FixedTopicRegisterTest extends PulsarTestSuiteBase { @Test - void listenEmptyTopics() { - TopicMetadataListener listener = new TopicMetadataListener(); + void listenEmptyTopics() throws IOException { + FixedTopicRegister listener = new FixedTopicRegister<>(emptyList()); SinkConfiguration configuration = sinkConfiguration(Duration.ofMinutes(5).toMillis()); TestProcessingTimeService timeService = new TestProcessingTimeService(); - List topics = listener.availableTopics(); + List topics = listener.topics(""); assertThat(topics).isEmpty(); listener.open(configuration, timeService); - topics = listener.availableTopics(); + topics = listener.topics(""); assertThat(topics).isEmpty(); + + listener.close(); } @Test @@ -60,41 +66,45 @@ void listenOnPartitions() throws Exception { operator().createTopic(topic, 6); List partitions = topicPartitions(topic, 6); - TopicMetadataListener listener = new TopicMetadataListener(partitions); + FixedTopicRegister listener = new FixedTopicRegister<>(partitions); long interval = Duration.ofMinutes(15).toMillis(); SinkConfiguration configuration = sinkConfiguration(interval); TestProcessingTimeService timeService = new TestProcessingTimeService(); - List topics = listener.availableTopics(); + List topics = listener.topics(""); assertEquals(topics, partitions); listener.open(configuration, timeService); - topics = listener.availableTopics(); + topics = listener.topics(""); assertEquals(topics, partitions); operator().increaseTopicPartitions(topic, 12); timeService.advance(interval); - topics = listener.availableTopics(); + topics = listener.topics(""); assertEquals(topics, partitions); + + listener.close(); } @Test - void fetchTopicPartitionInformation() { + void fetchTopicPartitionInformation() throws IOException { String topic = randomAlphabetic(10); operator().createTopic(topic, 8); - TopicMetadataListener listener = new TopicMetadataListener(singletonList(topic)); + FixedTopicRegister listener = new FixedTopicRegister<>(singletonList(topic)); SinkConfiguration configuration = sinkConfiguration(Duration.ofMinutes(10).toMillis()); TestProcessingTimeService timeService = new TestProcessingTimeService(); - List topics = listener.availableTopics(); + List topics = listener.topics(""); assertThat(topics).isEmpty(); listener.open(configuration, timeService); - topics = listener.availableTopics(); + topics = listener.topics(""); List desiredTopics = topicPartitions(topic, 8); assertThat(topics).hasSize(8).isEqualTo(desiredTopics); + + listener.close(); } @Test @@ -104,13 +114,13 @@ void fetchTopicPartitionUpdate() throws Exception { long interval = Duration.ofMinutes(20).toMillis(); - TopicMetadataListener listener = new TopicMetadataListener(singletonList(topic)); + FixedTopicRegister listener = new FixedTopicRegister<>(singletonList(topic)); SinkConfiguration configuration = sinkConfiguration(interval); TestProcessingTimeService timeService = new TestProcessingTimeService(); timeService.setCurrentTime(System.currentTimeMillis()); listener.open(configuration, timeService); - List topics = listener.availableTopics(); + List topics = listener.topics(""); List desiredTopics = topicPartitions(topic, 8); assertThat(topics).isEqualTo(desiredTopics); @@ -119,9 +129,29 @@ void fetchTopicPartitionUpdate() throws Exception { operator().increaseTopicPartitions(topic, 16); timeService.advance(interval); - topics = listener.availableTopics(); + topics = listener.topics(""); desiredTopics = topicPartitions(topic, 16); assertThat(topics).isEqualTo(desiredTopics); + + listener.close(); + } + + @Test + void fetchNonPartitionTopic() throws IOException { + String topic = randomAlphabetic(10); + operator().createTopic(topic, 0); + List nonPartitionTopic = Collections.singletonList(topicName(topic)); + + FixedTopicRegister listener = new FixedTopicRegister<>(nonPartitionTopic); + long interval = Duration.ofMinutes(15).toMillis(); + SinkConfiguration configuration = sinkConfiguration(interval); + TestProcessingTimeService timeService = new TestProcessingTimeService(); + + listener.open(configuration, timeService); + List topics = listener.topics(""); + assertEquals(topics, nonPartitionTopic); + + listener.close(); } private List topicPartitions(String topic, int partitionSize) { diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java index 9a72c8ace2a19..847c3dd5a9ea1 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java @@ -20,51 +20,31 @@ import org.apache.flink.connector.pulsar.testutils.PulsarTestContextFactory; import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; -import org.apache.flink.connector.pulsar.testutils.cases.MultipleTopicConsumingContext; -import org.apache.flink.connector.pulsar.testutils.cases.SingleTopicConsumingContext; import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntime; +import org.apache.flink.connector.pulsar.testutils.source.cases.MultipleTopicConsumingContext; +import org.apache.flink.connector.pulsar.testutils.source.cases.SingleTopicConsumingContext; import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment; -import org.apache.flink.connector.testframe.environment.TestEnvironment; -import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext; import org.apache.flink.connector.testframe.junit.annotations.TestContext; import org.apache.flink.connector.testframe.junit.annotations.TestEnv; import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem; import org.apache.flink.connector.testframe.junit.annotations.TestSemantics; import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase; import org.apache.flink.streaming.api.CheckpointingMode; +import org.apache.flink.test.junit5.MiniClusterExtension; -import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.extension.ExtendWith; /** Unite test class for {@link PulsarSource}. */ @SuppressWarnings("unused") +@ExtendWith(MiniClusterExtension.class) class PulsarSourceITCase extends SourceTestSuiteBase { - @Disabled // TODO: remove override after FLINK-26177 is fixed - @Override - public void testScaleUp( - TestEnvironment testEnv, - DataStreamSourceExternalContext externalContext, - CheckpointingMode semantic) - throws Exception { - super.testScaleUp(testEnv, externalContext, semantic); - } - - @Disabled // TODO: remove override after FLINK-26177 is fixed - @Override - public void testScaleDown( - TestEnvironment testEnv, - DataStreamSourceExternalContext externalContext, - CheckpointingMode semantic) - throws Exception { - super.testScaleDown(testEnv, externalContext, semantic); - } - // Defines test environment on Flink MiniCluster @TestEnv MiniClusterTestEnvironment flink = new MiniClusterTestEnvironment(); // Defines pulsar running environment @TestExternalSystem - PulsarTestEnvironment pulsar = new PulsarTestEnvironment(PulsarRuntime.embedded()); + PulsarTestEnvironment pulsar = new PulsarTestEnvironment(PulsarRuntime.mock()); @TestSemantics CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE}; diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumeratorTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumeratorTest.java index 1dcbe84ba61ee..aebb76119dfdc 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumeratorTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumeratorTest.java @@ -22,6 +22,8 @@ import org.apache.flink.api.connector.source.mocks.MockSplitEnumeratorContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.assigner.SplitAssigner; +import org.apache.flink.connector.pulsar.source.enumerator.assigner.SplitAssignerFactory; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.flink.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; @@ -51,6 +53,7 @@ import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_TYPE; import static org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor.latest; import static org.apache.flink.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber.getTopicPatternSubscriber; @@ -368,6 +371,7 @@ private PulsarSourceEnumerator createEnumerator( Configuration configuration = operator().config(); configuration.set(PULSAR_SUBSCRIPTION_TYPE, subscriptionType); + configuration.set(PULSAR_SUBSCRIPTION_NAME, randomAlphabetic(10)); if (enablePeriodicPartitionDiscovery) { configuration.set(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS, 60L); } else { @@ -375,15 +379,15 @@ private PulsarSourceEnumerator createEnumerator( } SourceConfiguration sourceConfiguration = new SourceConfiguration(configuration); - SplitsAssignmentState assignmentState = - new SplitsAssignmentState(latest(), sourceConfiguration, sourceEnumState); + SplitAssigner assigner = + SplitAssignerFactory.create(latest(), sourceConfiguration, sourceEnumState); return new PulsarSourceEnumerator( subscriber, StartCursor.earliest(), new FullRangeGenerator(), sourceConfiguration, enumContext, - assignmentState); + assigner); } private void registerReader( diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/SplitsAssignmentStateTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/SplitsAssignmentStateTest.java deleted file mode 100644 index ac811c3dddbfe..0000000000000 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/SplitsAssignmentStateTest.java +++ /dev/null @@ -1,119 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.connector.pulsar.source.enumerator; - -import org.apache.flink.api.connector.source.SplitsAssignment; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange; -import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; - -import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; -import org.apache.flink.shaded.guava30.com.google.common.collect.Sets; - -import org.apache.pulsar.client.api.SubscriptionType; -import org.junit.jupiter.api.Test; - -import java.util.List; -import java.util.Optional; -import java.util.Set; - -import static java.util.Collections.singletonList; -import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; -import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_TYPE; -import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.MAX_RANGE; -import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.createFullRange; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.InstanceOfAssertFactories.map; - -/** Unit tests for {@link SplitsAssignmentState}. */ -class SplitsAssignmentStateTest { - - private final Set partitions = - Sets.newHashSet( - new TopicPartition("some-topic", 1, new TopicRange(1, 30)), - new TopicPartition("some-topic", 2, new TopicRange(31, 60)), - new TopicPartition("some-topic", 3, new TopicRange(61, MAX_RANGE)), - new TopicPartition(randomAlphabetic(10), -1, createFullRange())); - - @Test - void assignSplitsForSharedSubscription() { - SplitsAssignmentState state1 = - new SplitsAssignmentState( - StopCursor.defaultStopCursor(), createConfig(SubscriptionType.Shared)); - state1.appendTopicPartitions(partitions); - Optional> assignment1 = - state1.assignSplits(Lists.newArrayList(0, 1, 2, 3, 4)); - - assertThat(assignment1) - .isPresent() - .get() - .extracting(SplitsAssignment::assignment) - .asInstanceOf(map(Integer.class, List.class)) - .hasSize(5) - .allSatisfy((idx, list) -> assertThat(list).hasSize(4)); - - Optional> assignment2 = - state1.assignSplits(Lists.newArrayList(0, 1, 2, 3, 4)); - assertThat(assignment2).isNotPresent(); - - // Reassign reader 3. - state1.putSplitsBackToPendingList(assignment1.get().assignment().get(3), 3); - Optional> assignment3 = - state1.assignSplits(Lists.newArrayList(0, 1, 2, 4)); - assertThat(assignment3).isNotPresent(); - - Optional> assignment4 = - state1.assignSplits(singletonList(3)); - assertThat(assignment4) - .isPresent() - .get() - .extracting(SplitsAssignment::assignment) - .asInstanceOf(map(Integer.class, List.class)) - .hasSize(1); - } - - @Test - void assignSplitsForExclusiveSubscription() { - SplitsAssignmentState state1 = - new SplitsAssignmentState( - StopCursor.defaultStopCursor(), createConfig(SubscriptionType.Exclusive)); - state1.appendTopicPartitions(partitions); - Optional> assignment1 = - state1.assignSplits(Lists.newArrayList(0, 1, 2, 3, 4)); - - assertThat(assignment1).isPresent(); - assertThat(assignment1.get().assignment()) - .hasSize(4) - .allSatisfy((idx, list) -> assertThat(list).hasSize(1)); - - Optional> assignment2 = - state1.assignSplits(Lists.newArrayList(0, 1, 2, 3, 4)); - assertThat(assignment2).isNotPresent(); - } - - private SourceConfiguration createConfig(SubscriptionType type) { - Configuration configuration = new Configuration(); - configuration.set(PULSAR_SUBSCRIPTION_TYPE, type); - - return new SourceConfiguration(configuration); - } -} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssignerTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssignerTest.java new file mode 100644 index 0000000000000..2e9ada3b74135 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssignerTest.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.source.enumerator.assigner; + +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + +import static java.util.Collections.singletonList; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** Unit tests for {@link NonSharedSplitAssigner}. */ +class NonSharedSplitAssignerTest extends SplitAssignerTestBase { + + @Test + void noMoreSplits() { + NonSharedSplitAssigner assigner = splitAssigner(true); + assertFalse(assigner.noMoreSplits(3)); + + assigner = splitAssigner(false); + assertFalse(assigner.noMoreSplits(3)); + + assigner.registerTopicPartitions(createPartitions("f", 8)); + assertFalse(assigner.noMoreSplits(3)); + + assigner.createAssignment(singletonList(1)); + assertTrue(assigner.noMoreSplits(1)); + assertTrue(assigner.noMoreSplits(3)); + } + + @Test + void partitionsAssignment() { + NonSharedSplitAssigner assigner = splitAssigner(true); + assigner.registerTopicPartitions(createPartitions("d", 4)); + List readers = Arrays.asList(1, 3, 5, 7); + + // Assignment with initial states. + Optional> assignment = + assigner.createAssignment(readers); + assertThat(assignment).isPresent(); + assertThat(assignment.get().assignment()).hasSize(1); + + // Reassignment with same readers. + assignment = assigner.createAssignment(readers); + assertThat(assignment).isNotPresent(); + + // Register new partition and assign. + assigner.registerTopicPartitions(createPartitions("e", 5)); + assigner.registerTopicPartitions(createPartitions("f", 1)); + assigner.registerTopicPartitions(createPartitions("g", 3)); + assigner.registerTopicPartitions(createPartitions("h", 4)); + assignment = assigner.createAssignment(readers); + assertThat(assignment).isPresent(); + assertThat(assignment.get().assignment()).hasSize(4); + + // Assign to new readers. + readers = Arrays.asList(2, 4, 6, 8); + assignment = assigner.createAssignment(readers); + assertThat(assignment).isNotPresent(); + } + + @Override + protected NonSharedSplitAssigner createAssigner( + StopCursor stopCursor, + SourceConfiguration sourceConfiguration, + PulsarSourceEnumState sourceEnumState) { + return new NonSharedSplitAssigner(stopCursor, sourceConfiguration, sourceEnumState); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssignerTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssignerTest.java new file mode 100644 index 0000000000000..91584b8768821 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssignerTest.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.source.enumerator.assigner; + +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + +import static java.util.Collections.singletonList; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** Unit tests for {@link SharedSplitAssigner}. */ +class SharedSplitAssignerTest extends SplitAssignerTestBase { + + @Test + void noMoreSplits() { + SharedSplitAssigner assigner = splitAssigner(true); + assertFalse(assigner.noMoreSplits(3)); + + assigner = splitAssigner(false); + assertFalse(assigner.noMoreSplits(3)); + + assigner.registerTopicPartitions(createPartitions("f", 8)); + assertFalse(assigner.noMoreSplits(3)); + + assigner.createAssignment(singletonList(1)); + assertTrue(assigner.noMoreSplits(1)); + assertFalse(assigner.noMoreSplits(3)); + + assigner.createAssignment(singletonList(3)); + assertTrue(assigner.noMoreSplits(3)); + } + + @Test + void partitionsAssignment() { + SharedSplitAssigner assigner = splitAssigner(true); + assigner.registerTopicPartitions(createPartitions("d", 4)); + List readers = Arrays.asList(1, 3, 5, 7); + + // Assignment with initial states. + Optional> assignment = + assigner.createAssignment(readers); + assertThat(assignment).isPresent(); + assertThat(assignment.get().assignment()).hasSize(4); + + // Reassignment with same readers. + assignment = assigner.createAssignment(readers); + assertThat(assignment).isNotPresent(); + + // Register new partition and assign. + assigner.registerTopicPartitions(createPartitions("e", 5)); + assignment = assigner.createAssignment(readers); + assertThat(assignment).isPresent(); + assertThat(assignment.get().assignment()).hasSize(4); + + // Assign to new readers. + readers = Arrays.asList(2, 4, 6, 8); + assignment = assigner.createAssignment(readers); + assertThat(assignment).isPresent(); + assertThat(assignment.get().assignment()) + .hasSize(4) + .allSatisfy((k, v) -> assertThat(v).hasSize(2)); + } + + @Override + protected SharedSplitAssigner createAssigner( + StopCursor stopCursor, + SourceConfiguration sourceConfiguration, + PulsarSourceEnumState sourceEnumState) { + return new SharedSplitAssigner(stopCursor, sourceConfiguration, sourceEnumState); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerTestBase.java new file mode 100644 index 0000000000000..6509401472068 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerTestBase.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.source.enumerator.assigner; + +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; +import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; +import org.apache.flink.util.TestLogger; + +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.Optional; +import java.util.Set; + +import static java.util.Collections.emptyList; +import static java.util.Collections.singleton; +import static java.util.Collections.singletonList; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS; +import static org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState.initialState; +import static org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor.defaultStopCursor; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.createFullRange; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test utils for split assigners. */ +abstract class SplitAssignerTestBase extends TestLogger { + + @Test + void registerTopicPartitionsWillOnlyReturnNewPartitions() { + T assigner = splitAssigner(true); + + Set partitions = createPartitions("persistent://public/default/a", 1); + List newPartitions = assigner.registerTopicPartitions(partitions); + assertThat(newPartitions) + .hasSize(1) + .first() + .hasFieldOrPropertyWithValue("topic", "persistent://public/default/a") + .hasFieldOrPropertyWithValue("partitionId", 1); + + newPartitions = assigner.registerTopicPartitions(partitions); + assertThat(newPartitions).isEmpty(); + + partitions = createPartitions("persistent://public/default/b", 2); + newPartitions = assigner.registerTopicPartitions(partitions); + assertThat(newPartitions) + .hasSize(1) + .hasSize(1) + .first() + .hasFieldOrPropertyWithValue("topic", "persistent://public/default/b") + .hasFieldOrPropertyWithValue("partitionId", 2); + } + + @Test + void noReadersProvideForAssignment() { + T assigner = splitAssigner(false); + assigner.registerTopicPartitions(createPartitions("c", 5)); + + Optional> assignment = + assigner.createAssignment(emptyList()); + assertThat(assignment).isNotPresent(); + } + + @Test + void noPartitionsProvideForAssignment() { + T assigner = splitAssigner(true); + Optional> assignment = + assigner.createAssignment(singletonList(4)); + assertThat(assignment).isNotPresent(); + } + + protected Set createPartitions(String topic, int partitionId) { + TopicPartition p1 = new TopicPartition(topic, partitionId, createFullRange()); + return singleton(p1); + } + + protected T splitAssigner(boolean discovery) { + Configuration configuration = new Configuration(); + + if (discovery) { + configuration.set(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS, 1000L); + } else { + configuration.set(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS, -1L); + } + + SourceConfiguration sourceConfiguration = new SourceConfiguration(configuration); + return createAssigner(defaultStopCursor(), sourceConfiguration, initialState()); + } + + protected abstract T createAssigner( + StopCursor stopCursor, + SourceConfiguration sourceConfiguration, + PulsarSourceEnumState sourceEnumState); +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java new file mode 100644 index 0000000000000..0632fa8f40294 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.source.enumerator.cursor; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; +import org.apache.flink.connector.pulsar.source.reader.split.PulsarOrderedPartitionSplitReader; +import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; +import org.apache.flink.connector.pulsar.testutils.PulsarTestSuiteBase; + +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Schema; +import org.junit.jupiter.api.Test; + +import java.io.IOException; + +import static java.util.Collections.singletonList; +import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; +import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_DEFAULT_FETCH_TIME; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_RECORDS; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_TIME; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.createFullRange; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test different implementation of StopCursor. */ +class StopCursorTest extends PulsarTestSuiteBase { + + @Test + void publishTimeStopCursor() throws IOException { + String topicName = randomAlphanumeric(5); + operator().createTopic(topicName, 2); + + PulsarOrderedPartitionSplitReader splitReader = + new PulsarOrderedPartitionSplitReader( + operator().client(), + operator().admin(), + sourceConfig(), + Schema.BYTES, + null); + // send the first message and set the stopCursor to filter any late stopCursor + operator() + .sendMessage( + topicNameWithPartition(topicName, 0), + Schema.STRING, + randomAlphanumeric(10)); + long currentTimeStamp = System.currentTimeMillis(); + TopicPartition partition = new TopicPartition(topicName, 0, createFullRange()); + PulsarPartitionSplit split = + new PulsarPartitionSplit( + partition, + StopCursor.atPublishTime(currentTimeStamp), + MessageId.earliest, + null); + SplitsAddition addition = new SplitsAddition<>(singletonList(split)); + splitReader.handleSplitsChanges(addition); + // first fetch should have result + RecordsWithSplitIds> firstResult = splitReader.fetch(); + assertThat(firstResult.nextSplit()).isNotNull(); + assertThat(firstResult.nextRecordFromSplit()).isNotNull(); + assertThat(firstResult.finishedSplits()).isEmpty(); + // send the second message and expect it will not be received + operator() + .sendMessage( + topicNameWithPartition(topicName, 0), + Schema.STRING, + randomAlphanumeric(10)); + RecordsWithSplitIds> secondResult = splitReader.fetch(); + assertThat(secondResult.nextSplit()).isNull(); + assertThat(secondResult.finishedSplits()).isNotEmpty(); + } + + private SourceConfiguration sourceConfig() { + Configuration config = operator().config(); + config.set(PULSAR_MAX_FETCH_RECORDS, 1); + config.set(PULSAR_DEFAULT_FETCH_TIME, 2000L); + config.set(PULSAR_MAX_FETCH_TIME, 3000L); + config.set(PULSAR_SUBSCRIPTION_NAME, randomAlphabetic(10)); + config.set(PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE, true); + return new SourceConfiguration(config); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaTest.java index 18888df64b52a..3478b5c235c0f 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaTest.java @@ -18,13 +18,21 @@ package org.apache.flink.connector.pulsar.source.reader.deserializer; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.serialization.SimpleStringSchema; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.connector.pulsar.SampleMessage.TestMessage; +import org.apache.flink.connector.pulsar.source.PulsarSource; +import org.apache.flink.connector.pulsar.source.PulsarSourceOptions; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; +import org.apache.flink.connector.pulsar.testutils.PulsarTestSuiteBase; import org.apache.flink.connector.testutils.source.deserialization.TestingDeserializationContext; import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.StringValue; +import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.Collector; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.function.FunctionWithException; @@ -33,9 +41,12 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.common.api.proto.MessageMetadata; +import org.apache.pulsar.common.schema.KeyValue; import org.junit.jupiter.api.Test; +import java.io.Serializable; import java.nio.ByteBuffer; +import java.util.Objects; import java.util.concurrent.ThreadLocalRandom; import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; @@ -44,13 +55,16 @@ import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.pulsarSchema; import static org.apache.flink.util.Preconditions.checkState; import static org.apache.pulsar.client.api.Schema.PROTOBUF_NATIVE; +import static org.apache.pulsar.client.api.SubscriptionType.Exclusive; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.mockito.Mockito.mock; /** Unit tests for {@link PulsarDeserializationSchema}. */ -class PulsarDeserializationSchemaTest { +class PulsarDeserializationSchemaTest extends PulsarTestSuiteBase { @Test void createFromFlinkDeserializationSchema() throws Exception { @@ -108,6 +122,243 @@ void createFromFlinkTypeInformation() throws Exception { assertEquals(collector.result, "test-content"); } + @Test + void primitiveStringPulsarSchema() { + final String topicName = + "primitiveString-" + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + operator().createTopic(topicName, 1); + String expectedMessage = randomAlphabetic(10); + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.STRING, + expectedMessage); + PulsarSource source = createSource(topicName, pulsarSchema(Schema.STRING)); + assertThatCode(() -> runPipeline(source, expectedMessage)).doesNotThrowAnyException(); + } + + @Test + void unversionedJsonStructPulsarSchema() { + final String topicName = + "unversionedJsonStruct-" + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + operator().createTopic(topicName, 1); + TestingUser expectedMessage = createRandomUser(); + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.JSON(TestingUser.class), + expectedMessage); + PulsarSource source = + createSource( + topicName, pulsarSchema(Schema.JSON(TestingUser.class), TestingUser.class)); + assertThatCode(() -> runPipeline(source, expectedMessage)).doesNotThrowAnyException(); + } + + @Test + void keyValueJsonStructPulsarSchema() { + final String topicName = + "keyValueJsonStruct-" + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + operator().createTopic(topicName, 1); + KeyValue expectedMessage = + new KeyValue<>(createRandomUser(), createRandomUser()); + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.KeyValue( + Schema.JSON(TestingUser.class), Schema.JSON(TestingUser.class)), + expectedMessage); + PulsarSource> source = + createSource( + topicName, + pulsarSchema( + Schema.KeyValue( + Schema.JSON(TestingUser.class), + Schema.JSON(TestingUser.class)), + TestingUser.class, + TestingUser.class)); + assertThatCode(() -> runPipeline(source, expectedMessage)).doesNotThrowAnyException(); + } + + @Test + void keyValueAvroStructPulsarSchema() { + final String topicName = + "keyValueAvroStruct-" + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + operator().createTopic(topicName, 1); + KeyValue expectedMessage = + new KeyValue<>(createRandomUser(), createRandomUser()); + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.KeyValue( + Schema.AVRO(TestingUser.class), Schema.AVRO(TestingUser.class)), + expectedMessage); + PulsarSource> source = + createSource( + topicName, + pulsarSchema( + Schema.KeyValue( + Schema.AVRO(TestingUser.class), + Schema.AVRO(TestingUser.class)), + TestingUser.class, + TestingUser.class)); + assertThatCode(() -> runPipeline(source, expectedMessage)).doesNotThrowAnyException(); + } + + @Test + void keyValuePrimitivePulsarSchema() { + final String topicName = + "keyValuePrimitive-" + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + operator().createTopic(topicName, 1); + KeyValue expectedMessage = new KeyValue<>(randomAlphabetic(5), 5); + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.KeyValue(Schema.STRING, Schema.INT32), + expectedMessage); + PulsarSource> source = + createSource( + topicName, + pulsarSchema( + Schema.KeyValue(Schema.STRING, Schema.INT32), + String.class, + Integer.class)); + assertThatCode(() -> runPipeline(source, expectedMessage)).doesNotThrowAnyException(); + } + + @Test + void keyValuePrimitiveKeyStructValuePulsarSchema() { + final String topicName = + "primitiveKeyStructValue-" + + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + operator().createTopic(topicName, 1); + KeyValue expectedMessage = + new KeyValue<>(randomAlphabetic(5), createRandomUser()); + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.KeyValue(Schema.STRING, Schema.JSON(TestingUser.class)), + expectedMessage); + PulsarSource> source = + createSource( + topicName, + pulsarSchema( + Schema.KeyValue(Schema.STRING, Schema.JSON(TestingUser.class)), + String.class, + TestingUser.class)); + assertThatCode(() -> runPipeline(source, expectedMessage)).doesNotThrowAnyException(); + } + + @Test + void keyValueStructKeyPrimitiveValuePulsarSchema() { + final String topicName = + "structKeyPrimitiveValue-" + + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + operator().createTopic(topicName, 1); + KeyValue expectedMessage = + new KeyValue<>(createRandomUser(), randomAlphabetic(5)); + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.KeyValue(Schema.JSON(TestingUser.class), Schema.STRING), + expectedMessage); + PulsarSource> source = + createSource( + topicName, + pulsarSchema( + Schema.KeyValue(Schema.JSON(TestingUser.class), Schema.STRING), + TestingUser.class, + String.class)); + assertThatCode(() -> runPipeline(source, expectedMessage)).doesNotThrowAnyException(); + } + + @Test + void simpleFlinkSchema() { + final String topicName = + "simpleFlinkSchema-" + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + operator().createTopic(topicName, 1); + String expectedMessage = randomAlphabetic(5); + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.STRING, + expectedMessage); + PulsarSource source = + createSource(topicName, flinkSchema(new SimpleStringSchema())); + assertThatCode(() -> runPipeline(source, expectedMessage)).doesNotThrowAnyException(); + } + + private PulsarSource createSource( + String topicName, PulsarDeserializationSchema deserializationSchema) { + return PulsarSource.builder() + .setDeserializationSchema(deserializationSchema) + .setServiceUrl(operator().serviceUrl()) + .setAdminUrl(operator().adminUrl()) + .setTopics(topicName) + .setSubscriptionType(Exclusive) + .setSubscriptionName(topicName + "-subscription") + .setBoundedStopCursor(StopCursor.latest()) + .setConfig(PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS, -1L) + .build(); + } + + private void runPipeline(PulsarSource source, T expected) throws Exception { + try (CloseableIterator iterator = + StreamExecutionEnvironment.getExecutionEnvironment() + .setParallelism(1) + .fromSource(source, WatermarkStrategy.noWatermarks(), "testSource") + .executeAndCollect()) { + assertThat(iterator).hasNext(); + assertThat(iterator.next()).isEqualTo(expected); + } + } + + /** A test POJO class. */ + public static class TestingUser implements Serializable { + private static final long serialVersionUID = -1123545861004770003L; + public String name; + public Integer age; + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public Integer getAge() { + return age; + } + + public void setAge(Integer age) { + this.age = age; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TestingUser that = (TestingUser) o; + return Objects.equals(name, that.name) && Objects.equals(age, that.age); + } + + @Override + public int hashCode() { + return Objects.hash(name, age); + } + } + + private TestingUser createRandomUser() { + TestingUser user = new TestingUser(); + user.setName(randomAlphabetic(5)); + user.setAge(ThreadLocalRandom.current().nextInt(0, Integer.MAX_VALUE)); + return user; + } + /** Create a test message by given bytes. The message don't contains any meta data. */ private Message getMessage( T message, FunctionWithException decoder) throws Exception { diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderTestBase.java index a42741d790076..e0720b7375c20 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderTestBase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderTestBase.java @@ -61,6 +61,7 @@ import java.util.stream.Stream; import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_DEFAULT_FETCH_TIME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_RECORDS; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_TIME; @@ -136,7 +137,8 @@ private PulsarSourceReaderBase sourceReader( boolean autoAcknowledgementEnabled, SubscriptionType subscriptionType) { Configuration configuration = operator().config(); configuration.set(PULSAR_MAX_FETCH_RECORDS, 1); - configuration.set(PULSAR_MAX_FETCH_TIME, 1000L); + configuration.set(PULSAR_DEFAULT_FETCH_TIME, 2000L); + configuration.set(PULSAR_MAX_FETCH_TIME, 3000L); configuration.set(PULSAR_SUBSCRIPTION_NAME, randomAlphabetic(10)); configuration.set(PULSAR_SUBSCRIPTION_TYPE, subscriptionType); if (autoAcknowledgementEnabled @@ -156,7 +158,7 @@ private PulsarSourceReaderBase sourceReader( SourceConfiguration sourceConfiguration = new SourceConfiguration(configuration); return (PulsarSourceReaderBase) PulsarSourceReaderFactory.create( - context, deserializationSchema, sourceConfiguration); + context, deserializationSchema, sourceConfiguration, null); } public class PulsarSourceReaderInvocationContextProvider diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReaderTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReaderTest.java index 3d58d5efeb4c8..4d93ce92ede36 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReaderTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReaderTest.java @@ -38,7 +38,7 @@ class PulsarOrderedPartitionSplitReaderTest extends PulsarPartitionSplitReaderTe @TestTemplate void consumeMessageCreatedBeforeHandleSplitsChangesWithoutSeek( - PulsarPartitionSplitReaderBase splitReader) { + PulsarPartitionSplitReaderBase splitReader) { String topicName = randomAlphabetic(10); operator().setupTopic(topicName, STRING, () -> randomAlphabetic(10)); handleSplit(splitReader, topicName, 0); @@ -47,7 +47,7 @@ void consumeMessageCreatedBeforeHandleSplitsChangesWithoutSeek( @TestTemplate void consumeMessageCreatedBeforeHandleSplitsChangesAndUseLatestStartCursorWithoutSeek( - PulsarPartitionSplitReaderBase splitReader) { + PulsarPartitionSplitReaderBase splitReader) { String topicName = randomAlphabetic(10); operator().setupTopic(topicName, STRING, () -> randomAlphabetic(10)); handleSplit(splitReader, topicName, 0, MessageId.latest); @@ -56,7 +56,7 @@ void consumeMessageCreatedBeforeHandleSplitsChangesAndUseLatestStartCursorWithou @TestTemplate void consumeMessageCreatedBeforeHandleSplitsChangesAndUseEarliestStartCursorWithoutSeek( - PulsarPartitionSplitReaderBase splitReader) { + PulsarPartitionSplitReaderBase splitReader) { String topicName = randomAlphabetic(10); operator().setupTopic(topicName, STRING, () -> randomAlphabetic(10)); handleSplit(splitReader, topicName, 0, MessageId.earliest); @@ -65,7 +65,7 @@ void consumeMessageCreatedBeforeHandleSplitsChangesAndUseEarliestStartCursorWith @TestTemplate void consumeMessageCreatedBeforeHandleSplitsChangesAndUseSecondLastMessageWithoutSeek( - PulsarPartitionSplitReaderBase splitReader) { + PulsarPartitionSplitReaderBase splitReader) { String topicName = randomAlphabetic(10); operator().setupTopic(topicName, STRING, () -> randomAlphabetic(10)); MessageIdImpl lastMessageId = diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java index 538e45826d7fc..96055cff72e2d 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java @@ -18,24 +18,22 @@ package org.apache.flink.connector.pulsar.source.reader.split; -import org.apache.flink.api.common.serialization.SimpleStringSchema; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; -import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.flink.connector.pulsar.testutils.PulsarTestSuiteBase; import org.apache.flink.connector.pulsar.testutils.extension.TestOrderlinessExtension; import org.apache.flink.util.TestLoggerExtension; -import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.impl.MessageIdImpl; import org.junit.jupiter.api.TestTemplate; @@ -62,14 +60,13 @@ import static java.util.Collections.singletonList; import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyAdmin; -import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyThrow; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_DEFAULT_FETCH_TIME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_RECORDS; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_TIME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.createFullRange; -import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.flinkSchema; import static org.apache.flink.connector.pulsar.testutils.PulsarTestCommonUtils.isAssignableFromParameterContext; import static org.apache.flink.connector.pulsar.testutils.extension.TestOrderlinessExtension.PULSAR_SOURCE_READER_SUBSCRIPTION_TYPE_STORE_KEY; import static org.apache.flink.connector.pulsar.testutils.extension.TestOrderlinessExtension.PULSAR_TEST_RESOURCE_NAMESPACE; @@ -86,7 +83,7 @@ TestOrderlinessExtension.class, TestLoggerExtension.class, }) -public abstract class PulsarPartitionSplitReaderTestBase extends PulsarTestSuiteBase { +abstract class PulsarPartitionSplitReaderTestBase extends PulsarTestSuiteBase { @RegisterExtension PulsarSplitReaderInvocationContextProvider provider = @@ -96,7 +93,8 @@ public abstract class PulsarPartitionSplitReaderTestBase extends PulsarTestSuite private SourceConfiguration sourceConfig() { Configuration config = operator().config(); config.set(PULSAR_MAX_FETCH_RECORDS, 1); - config.set(PULSAR_MAX_FETCH_TIME, 1000L); + config.set(PULSAR_DEFAULT_FETCH_TIME, 2000L); + config.set(PULSAR_MAX_FETCH_TIME, 3000L); config.set(PULSAR_SUBSCRIPTION_NAME, randomAlphabetic(10)); config.set(PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE, true); @@ -104,12 +102,12 @@ private SourceConfiguration sourceConfig() { } protected void handleSplit( - PulsarPartitionSplitReaderBase reader, String topicName, int partitionId) { + PulsarPartitionSplitReaderBase reader, String topicName, int partitionId) { handleSplit(reader, topicName, partitionId, null); } protected void handleSplit( - PulsarPartitionSplitReaderBase reader, + PulsarPartitionSplitReaderBase reader, String topicName, int partitionId, MessageId startPosition) { @@ -121,12 +119,12 @@ protected void handleSplit( } private void seekStartPositionAndHandleSplit( - PulsarPartitionSplitReaderBase reader, String topicName, int partitionId) { + PulsarPartitionSplitReaderBase reader, String topicName, int partitionId) { seekStartPositionAndHandleSplit(reader, topicName, partitionId, MessageId.latest); } private void seekStartPositionAndHandleSplit( - PulsarPartitionSplitReaderBase reader, + PulsarPartitionSplitReaderBase reader, String topicName, int partitionId, MessageId startPosition) { @@ -135,41 +133,57 @@ private void seekStartPositionAndHandleSplit( new PulsarPartitionSplit(partition, StopCursor.never(), null, null); SplitsAddition addition = new SplitsAddition<>(singletonList(split)); - // create consumer and seek before split changes - try (Consumer consumer = reader.createPulsarConsumer(partition)) { - // inclusive messageId - StartCursor startCursor = StartCursor.fromMessageId(startPosition); - startCursor.seekPosition(partition.getTopic(), partition.getPartitionId(), consumer); - } catch (PulsarClientException e) { - sneakyThrow(e); + // Create the subscription and set the start position for this reader. + // Remember not to use Consumer.seek(startPosition) + SourceConfiguration sourceConfiguration = reader.sourceConfiguration; + PulsarAdmin pulsarAdmin = reader.pulsarAdmin; + String subscriptionName = sourceConfiguration.getSubscriptionName(); + List subscriptions = + sneakyAdmin(() -> pulsarAdmin.topics().getSubscriptions(topicName)); + if (!subscriptions.contains(subscriptionName)) { + // If this subscription is not available. Just create it. + sneakyAdmin( + () -> + pulsarAdmin + .topics() + .createSubscription( + topicName, subscriptionName, startPosition)); + } else { + // Reset the subscription if this is existed. + sneakyAdmin( + () -> + pulsarAdmin + .topics() + .resetCursor(topicName, subscriptionName, startPosition)); } + // Accept the split and start consuming. reader.handleSplitsChanges(addition); } - private PulsarMessage fetchedMessage(PulsarPartitionSplitReaderBase splitReader) { + private Message fetchedMessage(PulsarPartitionSplitReaderBase splitReader) { return fetchedMessages(splitReader, 1, false).stream().findFirst().orElse(null); } - protected List> fetchedMessages( - PulsarPartitionSplitReaderBase splitReader, int expectedCount, boolean verify) { + protected List> fetchedMessages( + PulsarPartitionSplitReaderBase splitReader, int expectedCount, boolean verify) { return fetchedMessages( splitReader, expectedCount, verify, Boundedness.CONTINUOUS_UNBOUNDED); } - private List> fetchedMessages( - PulsarPartitionSplitReaderBase splitReader, + private List> fetchedMessages( + PulsarPartitionSplitReaderBase splitReader, int expectedCount, boolean verify, Boundedness boundedness) { - List> messages = new ArrayList<>(expectedCount); - List finishedSplits = new ArrayList<>(); + List> messages = new ArrayList<>(expectedCount); + List finishedSplits = new ArrayList<>(); for (int i = 0; i < 3; ) { try { - RecordsWithSplitIds> recordsBySplitIds = splitReader.fetch(); + RecordsWithSplitIds> recordsBySplitIds = splitReader.fetch(); if (recordsBySplitIds.nextSplit() != null) { // Collect the records in this split. - PulsarMessage record; + Message record; while ((record = recordsBySplitIds.nextRecordFromSplit()) != null) { messages.add(record); } @@ -185,7 +199,7 @@ private List> fetchedMessages( if (verify) { assertThat(messages).as("We should fetch the expected size").hasSize(expectedCount); if (boundedness == Boundedness.CONTINUOUS_UNBOUNDED) { - assertThat(finishedSplits).as("Split should not be marked as finished").hasSize(0); + assertThat(finishedSplits).as("Split should not be marked as finished").isEmpty(); } else { assertThat(finishedSplits).as("Split should be marked as finished").hasSize(1); } @@ -195,15 +209,15 @@ private List> fetchedMessages( } @TestTemplate - void pollMessageAfterTimeout(PulsarPartitionSplitReaderBase splitReader) + void pollMessageAfterTimeout(PulsarPartitionSplitReaderBase splitReader) throws InterruptedException, TimeoutException { String topicName = randomAlphabetic(10); // Add a split - seekStartPositionAndHandleSplit(splitReader, topicName, 0); + handleSplit(splitReader, topicName, 0); // Poll once with a null message - PulsarMessage message1 = fetchedMessage(splitReader); + Message message1 = fetchedMessage(splitReader); assertThat(message1).isNull(); // Send a message to pulsar @@ -213,7 +227,7 @@ void pollMessageAfterTimeout(PulsarPartitionSplitReaderBase splitReader) // Poll this message again waitUtil( () -> { - PulsarMessage message2 = fetchedMessage(splitReader); + Message message2 = fetchedMessage(splitReader); return message2 != null; }, ofSeconds(Integer.MAX_VALUE), @@ -222,16 +236,16 @@ void pollMessageAfterTimeout(PulsarPartitionSplitReaderBase splitReader) @TestTemplate void consumeMessageCreatedAfterHandleSplitChangesAndFetch( - PulsarPartitionSplitReaderBase splitReader) { + PulsarPartitionSplitReaderBase splitReader) { String topicName = randomAlphabetic(10); - seekStartPositionAndHandleSplit(splitReader, topicName, 0); + handleSplit(splitReader, topicName, 0); operator().sendMessage(topicNameWithPartition(topicName, 0), STRING, randomAlphabetic(10)); fetchedMessages(splitReader, 1, true); } @TestTemplate void consumeMessageCreatedBeforeHandleSplitsChanges( - PulsarPartitionSplitReaderBase splitReader) { + PulsarPartitionSplitReaderBase splitReader) { String topicName = randomAlphabetic(10); operator().setupTopic(topicName, STRING, () -> randomAlphabetic(10)); seekStartPositionAndHandleSplit(splitReader, topicName, 0); @@ -240,7 +254,7 @@ void consumeMessageCreatedBeforeHandleSplitsChanges( @TestTemplate void consumeMessageCreatedBeforeHandleSplitsChangesAndResetToEarliestPosition( - PulsarPartitionSplitReaderBase splitReader) { + PulsarPartitionSplitReaderBase splitReader) { String topicName = randomAlphabetic(10); operator().setupTopic(topicName, STRING, () -> randomAlphabetic(10)); seekStartPositionAndHandleSplit(splitReader, topicName, 0, MessageId.earliest); @@ -249,7 +263,7 @@ void consumeMessageCreatedBeforeHandleSplitsChangesAndResetToEarliestPosition( @TestTemplate void consumeMessageCreatedBeforeHandleSplitsChangesAndResetToLatestPosition( - PulsarPartitionSplitReaderBase splitReader) { + PulsarPartitionSplitReaderBase splitReader) { String topicName = randomAlphabetic(10); operator().setupTopic(topicName, STRING, () -> randomAlphabetic(10)); seekStartPositionAndHandleSplit(splitReader, topicName, 0, MessageId.latest); @@ -258,7 +272,7 @@ void consumeMessageCreatedBeforeHandleSplitsChangesAndResetToLatestPosition( @TestTemplate void consumeMessageCreatedBeforeHandleSplitsChangesAndUseSecondLastMessageIdCursor( - PulsarPartitionSplitReaderBase splitReader) { + PulsarPartitionSplitReaderBase splitReader) { String topicName = randomAlphabetic(10); operator().setupTopic(topicName, STRING, () -> randomAlphabetic(10)); @@ -284,7 +298,7 @@ void consumeMessageCreatedBeforeHandleSplitsChangesAndUseSecondLastMessageIdCurs } @TestTemplate - void emptyTopic(PulsarPartitionSplitReaderBase splitReader) { + void emptyTopic(PulsarPartitionSplitReaderBase splitReader) { String topicName = randomAlphabetic(10); operator().createTopic(topicName, DEFAULT_PARTITIONS); seekStartPositionAndHandleSplit(splitReader, topicName, 0); @@ -292,7 +306,7 @@ void emptyTopic(PulsarPartitionSplitReaderBase splitReader) { } @TestTemplate - void emptyTopicWithoutSeek(PulsarPartitionSplitReaderBase splitReader) { + void emptyTopicWithoutSeek(PulsarPartitionSplitReaderBase splitReader) { String topicName = randomAlphabetic(10); operator().createTopic(topicName, DEFAULT_PARTITIONS); handleSplit(splitReader, topicName, 0); @@ -300,8 +314,7 @@ void emptyTopicWithoutSeek(PulsarPartitionSplitReaderBase splitReader) { } @TestTemplate - void wakeupSplitReaderShouldNotCauseException( - PulsarPartitionSplitReaderBase splitReader) { + void wakeupSplitReaderShouldNotCauseException(PulsarPartitionSplitReaderBase splitReader) { handleSplit(splitReader, "non-exist", 0); AtomicReference error = new AtomicReference<>(); Thread t = @@ -324,24 +337,22 @@ void wakeupSplitReaderShouldNotCauseException( } @TestTemplate - void assignNoSplits(PulsarPartitionSplitReaderBase splitReader) { + void assignNoSplits(PulsarPartitionSplitReaderBase splitReader) { assertThat(fetchedMessage(splitReader)).isNull(); } /** Create a split reader with max message 1, fetch timeout 1s. */ - private PulsarPartitionSplitReaderBase splitReader(SubscriptionType subscriptionType) { + private PulsarPartitionSplitReaderBase splitReader(SubscriptionType subscriptionType) { if (subscriptionType == SubscriptionType.Failover) { - return new PulsarOrderedPartitionSplitReader<>( - operator().client(), - operator().admin(), - sourceConfig(), - flinkSchema(new SimpleStringSchema())); + return new PulsarOrderedPartitionSplitReader( + operator().client(), operator().admin(), sourceConfig(), Schema.BYTES, null); } else { - return new PulsarUnorderedPartitionSplitReader<>( + return new PulsarUnorderedPartitionSplitReader( operator().client(), operator().admin(), sourceConfig(), - flinkSchema(new SimpleStringSchema()), + Schema.BYTES, + null, null); } } @@ -370,9 +381,9 @@ public Stream provideTestTemplateInvocationContex public static class PulsarSplitReaderInvocationContext implements TestTemplateInvocationContext { - private final PulsarPartitionSplitReaderBase splitReader; + private final PulsarPartitionSplitReaderBase splitReader; - public PulsarSplitReaderInvocationContext(PulsarPartitionSplitReaderBase splitReader) { + public PulsarSplitReaderInvocationContext(PulsarPartitionSplitReaderBase splitReader) { this.splitReader = checkNotNull(splitReader); } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java new file mode 100644 index 0000000000000..0d6a59f34e798 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java @@ -0,0 +1,431 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.pulsar.sink.PulsarSink; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; +import org.apache.flink.connector.pulsar.source.PulsarSource; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.table.sink.PulsarTableSerializationSchemaFactory; +import org.apache.flink.connector.pulsar.table.sink.PulsarTableSink; +import org.apache.flink.connector.pulsar.table.source.PulsarTableDeserializationSchemaFactory; +import org.apache.flink.connector.pulsar.table.source.PulsarTableSource; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.WatermarkSpec; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkV2Provider; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.connector.source.SourceProvider; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.expressions.utils.ResolvedExpressionMock; +import org.apache.flink.table.factories.TestFormatFactory; +import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; +import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; +import org.apache.flink.table.types.DataType; + +import org.apache.pulsar.client.api.SubscriptionType; +import org.assertj.core.util.Lists; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL; +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_SERVICE_URL; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.ADMIN_URL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FORMAT; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SERVICE_URL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_NAME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; +import static org.apache.flink.table.factories.FactoryUtil.FORMAT; +import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; +import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * This test aims to verify that {@link PulsarTableFactory} can consume proper config options and + * produce expected {@link PulsarTableSource} and {@link PulsarTableSink}. It guarantees that config + * options is used internally by the implementation classes. + */ +public class PulsarTableFactoryTest { + private static final String TEST_TOPIC = "test-topic"; + private static final String TEST_ADMIN_URL = "http://my-broker.example.com:8080"; + private static final String TEST_SERVICE_URL = "pulsar://localhost:6650"; + private static final String TEST_SUBSCRIPTION_NAME = "default-subscription"; + + private static final String NAME = "name"; + private static final String COUNT = "count"; + private static final String TIME = "time"; + private static final String METADATA = "metadata"; + private static final String WATERMARK_EXPRESSION = TIME + " - INTERVAL '5' SECOND"; + private static final DataType WATERMARK_DATATYPE = DataTypes.TIMESTAMP(3); + private static final String COMPUTED_COLUMN_NAME = "computed-column"; + private static final String COMPUTED_COLUMN_EXPRESSION = COUNT + " + 1.0"; + private static final DataType COMPUTED_COLUMN_DATATYPE = DataTypes.DECIMAL(10, 3); + + private static final Properties EXPECTED_PULSAR_SOURCE_PROPERTIES = new Properties(); + private static final Properties EXPECTED_PULSAR_SINK_PROPERTIES = new Properties(); + + private static final String FORMAT_DELIMITER_KEY = + String.format("%s.%s", TestFormatFactory.IDENTIFIER, TestFormatFactory.DELIMITER.key()); + + private static final String FORMAT_FAIL_ON_MISSING_KEY = + String.format( + "%s.%s", TestFormatFactory.IDENTIFIER, TestFormatFactory.FAIL_ON_MISSING.key()); + + static { + EXPECTED_PULSAR_SOURCE_PROPERTIES.setProperty(PULSAR_ADMIN_URL.key(), TEST_ADMIN_URL); + EXPECTED_PULSAR_SOURCE_PROPERTIES.setProperty(PULSAR_SERVICE_URL.key(), TEST_SERVICE_URL); + EXPECTED_PULSAR_SOURCE_PROPERTIES.setProperty( + PULSAR_SUBSCRIPTION_NAME.key(), TEST_SUBSCRIPTION_NAME); + + EXPECTED_PULSAR_SINK_PROPERTIES.setProperty(PULSAR_ADMIN_URL.key(), TEST_ADMIN_URL); + EXPECTED_PULSAR_SINK_PROPERTIES.setProperty(PULSAR_SERVICE_URL.key(), TEST_SERVICE_URL); + } + + private static final ResolvedSchema SCHEMA = + new ResolvedSchema( + Arrays.asList( + Column.physical(NAME, DataTypes.STRING().notNull()), + Column.physical(COUNT, DataTypes.DECIMAL(38, 18)), + Column.physical(TIME, DataTypes.TIMESTAMP(3)), + Column.computed( + COMPUTED_COLUMN_NAME, + ResolvedExpressionMock.of( + COMPUTED_COLUMN_DATATYPE, COMPUTED_COLUMN_EXPRESSION))), + Collections.singletonList( + WatermarkSpec.of( + TIME, + ResolvedExpressionMock.of( + WATERMARK_DATATYPE, WATERMARK_EXPRESSION))), + null); + + private static final ResolvedSchema SCHEMA_WITH_METADATA = + new ResolvedSchema( + Arrays.asList( + Column.physical(NAME, DataTypes.STRING()), + Column.physical(COUNT, DataTypes.DECIMAL(38, 18)), + Column.metadata(TIME, DataTypes.TIMESTAMP(3), "publish_time", false), + Column.metadata( + METADATA, DataTypes.STRING(), "value.metadata_2", false)), + Collections.emptyList(), + null); + + private static final DataType SCHEMA_DATA_TYPE = SCHEMA.toPhysicalRowDataType(); + + @Test + public void testTableSource() { + final Map configuration = getBasicSourceOptions(); + final PulsarTableSource actualPulsarSource = + (PulsarTableSource) createTableSource(SCHEMA, configuration); + + final DecodingFormat> valueDecodingFormat = + new TestFormatFactory.DecodingFormatMock(",", true); + + final PulsarTableDeserializationSchemaFactory deserializationSchemaFactory = + new PulsarTableDeserializationSchemaFactory( + SCHEMA_DATA_TYPE, + null, + new int[0], + valueDecodingFormat, + new int[] {0, 1, 2}); + + final PulsarTableSource expectedPulsarSource = + new PulsarTableSource( + deserializationSchemaFactory, + valueDecodingFormat, + Lists.list(TEST_TOPIC), + EXPECTED_PULSAR_SOURCE_PROPERTIES, + StartCursor.earliest(), + StopCursor.never(), + SubscriptionType.Exclusive); + assertThat(actualPulsarSource).isEqualTo(expectedPulsarSource); + + ScanTableSource.ScanRuntimeProvider provider = + actualPulsarSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); + assertPulsarSourceIsSameAsExpected(provider); + } + + @Test + public void testTableSourceWithKeyValue() { + final Map configuration = getSourceKeyValueOptions(); + + final PulsarTableSource actualPulsarSource = + (PulsarTableSource) createTableSource(SCHEMA, configuration); + // initialize stateful testing formats + actualPulsarSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); + + final TestFormatFactory.DecodingFormatMock keyDecodingFormat = + new TestFormatFactory.DecodingFormatMock("#", false); + keyDecodingFormat.producedDataType = + DataTypes.ROW(DataTypes.FIELD(NAME, DataTypes.STRING().notNull())).notNull(); + + final TestFormatFactory.DecodingFormatMock valueDecodingFormat = + new TestFormatFactory.DecodingFormatMock("|", false); + valueDecodingFormat.producedDataType = + DataTypes.ROW( + DataTypes.FIELD(COUNT, DataTypes.DECIMAL(38, 18)), + DataTypes.FIELD(TIME, DataTypes.TIMESTAMP(3))) + .notNull(); + + final PulsarTableDeserializationSchemaFactory deserializationSchemaFactory = + new PulsarTableDeserializationSchemaFactory( + SCHEMA_DATA_TYPE, + keyDecodingFormat, + new int[] {0}, + valueDecodingFormat, + new int[] {1, 2}); + + final PulsarTableSource expectedPulsarSource = + new PulsarTableSource( + deserializationSchemaFactory, + valueDecodingFormat, + Lists.list(TEST_TOPIC), + EXPECTED_PULSAR_SOURCE_PROPERTIES, + StartCursor.earliest(), + StopCursor.never(), + SubscriptionType.Exclusive); + + assertThat(actualPulsarSource).isEqualTo(expectedPulsarSource); + } + + @Test + public void testTableSourceWithKeyValueAndMetadata() { + final Map options = getSourceKeyValueOptions(); + options.put("test-format.readable-metadata", "metadata_1:INT, metadata_2:STRING"); + + final PulsarTableSource actualPulsarSource = + (PulsarTableSource) createTableSource(SCHEMA_WITH_METADATA, options); + // initialize stateful testing formats + actualPulsarSource.applyReadableMetadata( + Arrays.asList("publish_time", "value.metadata_2"), + SCHEMA_WITH_METADATA.toSourceRowDataType()); + actualPulsarSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); + + final TestFormatFactory.DecodingFormatMock expectedKeyFormat = + new TestFormatFactory.DecodingFormatMock( + "#", false, ChangelogMode.insertOnly(), Collections.emptyMap()); + expectedKeyFormat.producedDataType = + DataTypes.ROW(DataTypes.FIELD(NAME, DataTypes.STRING())).notNull(); + + final Map expectedReadableMetadata = new HashMap<>(); + expectedReadableMetadata.put("metadata_1", DataTypes.INT()); + expectedReadableMetadata.put("metadata_2", DataTypes.STRING()); + + final TestFormatFactory.DecodingFormatMock expectedValueFormat = + new TestFormatFactory.DecodingFormatMock( + "|", false, ChangelogMode.insertOnly(), expectedReadableMetadata); + expectedValueFormat.producedDataType = + DataTypes.ROW( + DataTypes.FIELD(COUNT, DataTypes.DECIMAL(38, 18)), + DataTypes.FIELD("metadata_2", DataTypes.STRING())) + .notNull(); + expectedValueFormat.metadataKeys = Collections.singletonList("metadata_2"); + + final PulsarTableDeserializationSchemaFactory deserializationSchemaFactory = + new PulsarTableDeserializationSchemaFactory( + SCHEMA_WITH_METADATA.toPhysicalRowDataType(), + expectedKeyFormat, + new int[] {0}, + expectedValueFormat, + new int[] {1}); + + final PulsarTableSource expectedPulsarSource = + new PulsarTableSource( + deserializationSchemaFactory, + expectedValueFormat, + Lists.list(TEST_TOPIC), + EXPECTED_PULSAR_SOURCE_PROPERTIES, + StartCursor.earliest(), + StopCursor.never(), + SubscriptionType.Exclusive); + + deserializationSchemaFactory.setProducedDataType( + SCHEMA_WITH_METADATA.toSourceRowDataType()); + deserializationSchemaFactory.setConnectorMetadataKeys( + Collections.singletonList("publish_time")); + + assertThat(actualPulsarSource).isEqualTo(expectedPulsarSource); + } + + @Test + public void testTableSink() { + final Map modifiedOptions = getBasicSinkOptions(); + final DynamicTableSink actualPulsarTableSink = createTableSink(SCHEMA, modifiedOptions); + + final EncodingFormat> valueEncodingFormat = + new TestFormatFactory.EncodingFormatMock(","); + + final PulsarTableSerializationSchemaFactory serializationSchemaFactory = + new PulsarTableSerializationSchemaFactory( + SCHEMA_DATA_TYPE, + null, + new int[0], + valueEncodingFormat, + new int[] {0, 1, 2}); + + final PulsarTableSink expectedPulsarTableSink = + new PulsarTableSink( + serializationSchemaFactory, + valueEncodingFormat.getChangelogMode(), + Lists.list(TEST_TOPIC), + EXPECTED_PULSAR_SINK_PROPERTIES, + DeliveryGuarantee.AT_LEAST_ONCE, + null, + TopicRoutingMode.ROUND_ROBIN, + 0); + assertThat(actualPulsarTableSink).isEqualTo(expectedPulsarTableSink); + + DynamicTableSink.SinkRuntimeProvider provider = + actualPulsarTableSink.getSinkRuntimeProvider(new SinkRuntimeProviderContext(false)); + assertThat(provider).isInstanceOf(SinkV2Provider.class); + final SinkV2Provider sinkProvider = (SinkV2Provider) provider; + final Sink sinkFunction = sinkProvider.createSink(); + assertThat(sinkFunction).isInstanceOf(PulsarSink.class); + } + + @Test + public void testTableSinkWithKeyValue() { + final Map modifiedOptions = getSinkKeyValueOptions(); + final PulsarTableSink actualPulsarTableSink = + (PulsarTableSink) createTableSink(SCHEMA, modifiedOptions); + // initialize stateful testing formats + actualPulsarTableSink.getSinkRuntimeProvider(new SinkRuntimeProviderContext(false)); + + final TestFormatFactory.EncodingFormatMock keyEncodingFormat = + new TestFormatFactory.EncodingFormatMock("#"); + keyEncodingFormat.consumedDataType = + DataTypes.ROW(DataTypes.FIELD(NAME, DataTypes.STRING().notNull())).notNull(); + + final TestFormatFactory.EncodingFormatMock valueEncodingFormat = + new TestFormatFactory.EncodingFormatMock("|"); + valueEncodingFormat.consumedDataType = + DataTypes.ROW( + DataTypes.FIELD(COUNT, DataTypes.DECIMAL(38, 18)), + DataTypes.FIELD(TIME, DataTypes.TIMESTAMP(3))) + .notNull(); + + final PulsarTableSerializationSchemaFactory serializationSchemaFactory = + new PulsarTableSerializationSchemaFactory( + SCHEMA_DATA_TYPE, + keyEncodingFormat, + new int[] {0}, + valueEncodingFormat, + new int[] {1, 2}); + + final PulsarTableSink expectedPulsarTableSink = + new PulsarTableSink( + serializationSchemaFactory, + valueEncodingFormat.getChangelogMode(), + Lists.list(TEST_TOPIC), + EXPECTED_PULSAR_SINK_PROPERTIES, + DeliveryGuarantee.AT_LEAST_ONCE, + null, + TopicRoutingMode.ROUND_ROBIN, + 0); + assertThat(actualPulsarTableSink).isEqualTo(expectedPulsarTableSink); + } + + private static Map getBasicSourceOptions() { + Map tableOptions = new HashMap<>(); + tableOptions.put(CONNECTOR.key(), PulsarTableFactory.IDENTIFIER); + tableOptions.put(TOPICS.key(), TEST_TOPIC); + tableOptions.put(ADMIN_URL.key(), TEST_ADMIN_URL); + tableOptions.put(SERVICE_URL.key(), TEST_SERVICE_URL); + tableOptions.put(SOURCE_SUBSCRIPTION_NAME.key(), TEST_SUBSCRIPTION_NAME); + // Format options. + tableOptions.put(FORMAT.key(), TestFormatFactory.IDENTIFIER); + tableOptions.put(FORMAT_DELIMITER_KEY, ","); + tableOptions.put(FORMAT_FAIL_ON_MISSING_KEY, "true"); + return tableOptions; + } + + private static Map getSourceKeyValueOptions() { + Map tableOptions = new HashMap<>(); + tableOptions.put(CONNECTOR.key(), PulsarTableFactory.IDENTIFIER); + tableOptions.put(TOPICS.key(), TEST_TOPIC); + tableOptions.put(ADMIN_URL.key(), TEST_ADMIN_URL); + tableOptions.put(SERVICE_URL.key(), TEST_SERVICE_URL); + tableOptions.put(SOURCE_SUBSCRIPTION_NAME.key(), TEST_SUBSCRIPTION_NAME); + // Format options. + tableOptions.put(FORMAT.key(), TestFormatFactory.IDENTIFIER); + tableOptions.put(FORMAT_DELIMITER_KEY, "|"); + tableOptions.put(FORMAT_FAIL_ON_MISSING_KEY, "false"); + + tableOptions.put(KEY_FORMAT.key(), TestFormatFactory.IDENTIFIER); + tableOptions.put("key." + FORMAT_DELIMITER_KEY, "#"); + tableOptions.put("key." + FORMAT_FAIL_ON_MISSING_KEY, "false"); + tableOptions.put(KEY_FIELDS.key(), NAME); + return tableOptions; + } + + private static Map getBasicSinkOptions() { + Map tableOptions = new HashMap<>(); + tableOptions.put(CONNECTOR.key(), PulsarTableFactory.IDENTIFIER); + tableOptions.put(TOPICS.key(), TEST_TOPIC); + tableOptions.put(ADMIN_URL.key(), TEST_ADMIN_URL); + tableOptions.put(SERVICE_URL.key(), TEST_SERVICE_URL); + // Format options. + tableOptions.put(FORMAT.key(), TestFormatFactory.IDENTIFIER); + tableOptions.put(FORMAT_DELIMITER_KEY, ","); + tableOptions.put(FORMAT_FAIL_ON_MISSING_KEY, "true"); + return tableOptions; + } + + private static Map getSinkKeyValueOptions() { + Map tableOptions = new HashMap<>(); + tableOptions.put(CONNECTOR.key(), PulsarTableFactory.IDENTIFIER); + tableOptions.put(TOPICS.key(), TEST_TOPIC); + tableOptions.put(ADMIN_URL.key(), TEST_ADMIN_URL); + tableOptions.put(SERVICE_URL.key(), TEST_SERVICE_URL); + // Format options. + tableOptions.put(FORMAT.key(), TestFormatFactory.IDENTIFIER); + tableOptions.put(FORMAT_DELIMITER_KEY, "|"); + tableOptions.put(FORMAT_FAIL_ON_MISSING_KEY, "false"); + + tableOptions.put(KEY_FORMAT.key(), TestFormatFactory.IDENTIFIER); + tableOptions.put("key." + FORMAT_DELIMITER_KEY, "#"); + tableOptions.put("key." + FORMAT_FAIL_ON_MISSING_KEY, "false"); + tableOptions.put(KEY_FIELDS.key(), NAME); + return tableOptions; + } + + private PulsarSource assertPulsarSourceIsSameAsExpected( + ScanTableSource.ScanRuntimeProvider provider) { + assertThat(provider).isInstanceOf(SourceProvider.class); + final SourceProvider sourceProvider = (SourceProvider) provider; + return (PulsarSource) sourceProvider.createSource(); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java new file mode 100644 index 0000000000000..5dfe450195ab6 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java @@ -0,0 +1,644 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table; + +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.connector.pulsar.table.testutils.TestingUser; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.test.util.SuccessException; +import org.apache.flink.types.Row; + +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.junit.jupiter.params.provider.ValueSource; + +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Stream; + +import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; +import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; +import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyThrow; +import static org.apache.flink.connector.pulsar.table.testutils.PulsarTableTestUtils.collectRows; +import static org.apache.flink.connector.pulsar.table.testutils.TestingUser.createRandomUser; +import static org.apache.flink.util.CollectionUtil.entry; +import static org.apache.flink.util.CollectionUtil.map; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; +import static org.assertj.core.api.Assertions.assertThatNoException; + +/** + * IT cases for the Pulsar table source and sink. It aims to verify runtime behaviour and certain + * use cases are correct and can produce/consume the desired records as user specifies. + */ +@ExtendWith(MiniClusterExtension.class) +public class PulsarTableITCase extends PulsarTableTestBase { + + private static final String JSON_FORMAT = "json"; + private static final String AVRO_FORMAT = "avro"; + private static final String CSV_FORMAT = "csv"; + private static final String RAW_FORMAT = "raw"; + + @ParameterizedTest + @ValueSource(strings = {JSON_FORMAT, AVRO_FORMAT, CSV_FORMAT}) + void pulsarSourceSink(String format) throws Exception { + // we always use a different topic name for each parameterized topic, + // in order to make sure the topic can be created. + final String topic = "test_topic_" + format + randomAlphanumeric(3); + createTestTopic(topic, 1); + + // ---------- Produce an event time stream into Pulsar ------------------- + String randomTableName = randomAlphabetic(5); + final String createTable = + String.format( + "create table %s (\n" + + " `computed-price` as price + 1.0,\n" + + " price decimal(38, 18),\n" + + " currency string,\n" + + " log_date date,\n" + + " log_time time(3),\n" + + " log_ts timestamp(3),\n" + + " ts as log_ts + INTERVAL '1' SECOND,\n" + + " watermark for ts as ts\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'format' = '%s'\n" + + ")", + randomTableName, + PulsarTableFactory.IDENTIFIER, + topic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + format); + + tableEnv.executeSql(createTable); + + String initialValues = + String.format( + "INSERT INTO %s\n" + + "SELECT CAST(price AS DECIMAL(10, 2)), currency, " + + " CAST(d AS DATE), CAST(t AS TIME(0)), CAST(ts AS TIMESTAMP(3))\n" + + "FROM (VALUES (2.02,'Euro','2019-12-12', '00:00:01', '2019-12-12 00:00:01.001001'), \n" + + " (1.11,'US Dollar','2019-12-12', '00:00:02', '2019-12-12 00:00:02.002001'), \n" + + " (50,'Yen','2019-12-12', '00:00:03', '2019-12-12 00:00:03.004001'), \n" + + " (3.1,'Euro','2019-12-12', '00:00:04', '2019-12-12 00:00:04.005001'), \n" + + " (5.33,'US Dollar','2019-12-12', '00:00:05', '2019-12-12 00:00:05.006001'), \n" + + " (0,'DUMMY','2019-12-12', '00:00:10', '2019-12-12 00:00:10'))\n" + + " AS orders (price, currency, d, t, ts)", + randomTableName); + tableEnv.executeSql(initialValues).await(); + + String query = + String.format( + "SELECT\n" + + " CAST(TUMBLE_END(ts, INTERVAL '5' SECOND) AS VARCHAR),\n" + + " CAST(MAX(log_date) AS VARCHAR),\n" + + " CAST(MAX(log_time) AS VARCHAR),\n" + + " CAST(MAX(ts) AS VARCHAR),\n" + + " COUNT(*),\n" + + " CAST(MAX(price) AS DECIMAL(10, 2))\n" + + "FROM %s\n" + + "GROUP BY TUMBLE(ts, INTERVAL '5' SECOND)", + randomTableName); + + DataStream result = tableEnv.toDataStream(tableEnv.sqlQuery(query)); + TestingSinkFunction sink = new TestingSinkFunction(2); + result.addSink(sink).setParallelism(1); + + try { + env.execute("Job_2"); + } catch (Throwable e) { + if (!isCausedByJobFinished(e)) { + // re-throw + throw e; + } + } + + List expected = + Arrays.asList( + "+I[2019-12-12 00:00:05.000, 2019-12-12, 00:00:03, 2019-12-12 00:00:04.004, 3, 50.00]", + "+I[2019-12-12 00:00:10.000, 2019-12-12, 00:00:05, 2019-12-12 00:00:06.006, 2, 5.33]"); + + assertThat(TestingSinkFunction.rows).isEqualTo(expected); + } + + @ParameterizedTest + @ValueSource(strings = {JSON_FORMAT, CSV_FORMAT}) + void pulsarSourceSinkWithKeyAndPartialValue(String format) throws Exception { + // we always use a different topic name for each parameterized topic, + // in order to make sure the topic can be created. + final String topic = "key_partial_value_topic_" + format + randomAlphanumeric(3); + createTestTopic(topic, 1); + + // ---------- Produce an event time stream into Pulsar ------------------- + + // k_user_id and user_id have different data types to verify the correct mapping, + // fields are reordered on purpose + String randomTableName = randomAlphabetic(5); + final String createTable = + String.format( + "CREATE TABLE %s (\n" + + " `user_id` BIGINT,\n" + + " `name` STRING,\n" + + " `event_id` BIGINT,\n" + + " `payload` STRING\n" + + ") WITH (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'format' = '%s',\n" + + " 'key.format' = '%s',\n" + + " 'key.fields' = 'user_id; event_id'\n" + + ")", + randomTableName, + PulsarTableFactory.IDENTIFIER, + topic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + format, + format); + + tableEnv.executeSql(createTable); + + String initialValues = + String.format( + "INSERT INTO %s\n" + + "VALUES\n" + + " (1, 'name 1', 100, 'payload 1'),\n" + + " (2, 'name 2', 101, 'payload 2'),\n" + + " (3, 'name 3', 102, 'payload 3')", + randomTableName); + tableEnv.executeSql(initialValues).await(); + + final List result = + collectRows( + tableEnv.sqlQuery(String.format("SELECT * FROM %s", randomTableName)), 3); + + assertThat(result) + .containsExactlyInAnyOrder( + Row.of(1L, "name 1", 100L, "payload 1"), + Row.of(2L, "name 2", 101L, "payload 2"), + Row.of(3L, "name 3", 102L, "payload 3")); + } + + @ParameterizedTest + @ValueSource(strings = {JSON_FORMAT, AVRO_FORMAT, CSV_FORMAT}) + void pulsarSourceSinkWithMetadata(String format) throws Exception { + // we always use a different topic name for each parameterized topic, + // in order to make sure the topic can be created. + final String topic = "metadata_topic_" + format + randomAlphanumeric(3); + createTestTopic(topic, 1); + + String randomTableName = randomAlphabetic(5); + final String createTable = + String.format( + "CREATE TABLE %s (\n" + + " `physical_1` STRING,\n" + + " `physical_2` INT,\n" + + " `event_time` TIMESTAMP(3) METADATA,\n" + + " `properties` MAP METADATA,\n" + + " `physical_3` BOOLEAN\n" + + ") WITH (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'pulsar.producer.producerName' = 'pulsar-table-test',\n" + + " 'format' = '%s'\n" + + ")", + randomTableName, + PulsarTableFactory.IDENTIFIER, + topic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + format); + tableEnv.executeSql(createTable); + + String initialValues = + String.format( + "INSERT INTO %s\n" + + "VALUES\n" + + " ('data 1', 1, TIMESTAMP '2022-03-24 13:12:11.123', MAP['k1', 'C0FFEE', 'k2', 'BABE01'], TRUE),\n" + + " ('data 2', 2, TIMESTAMP '2022-03-25 13:12:11.123', CAST(NULL AS MAP), FALSE),\n" + + " ('data 3', 3, TIMESTAMP '2022-03-26 13:12:11.123', MAP['k1', 'C0FFEE', 'k2', 'BABE01'], TRUE)", + randomTableName); + tableEnv.executeSql(initialValues).await(); + + // ---------- Consume stream from Pulsar ------------------- + + final List result = + collectRows( + tableEnv.sqlQuery(String.format("SELECT * FROM %s", randomTableName)), 3); + assertThat(result) + .containsExactlyInAnyOrder( + Row.of( + "data 1", + 1, + LocalDateTime.parse("2022-03-24T13:12:11.123"), + map(entry("k1", "C0FFEE"), entry("k2", "BABE01")), + true), + Row.of( + "data 2", + 2, + LocalDateTime.parse("2022-03-25T13:12:11.123"), + Collections.emptyMap(), + false), + Row.of( + "data 3", + 3, + LocalDateTime.parse("2022-03-26T13:12:11.123"), + map(entry("k1", "C0FFEE"), entry("k2", "BABE01")), + true)); + } + + @Test + void sendMessageWithPropertiesAndReadPropertiesMetadata() throws Exception { + final String sourceTopic = "source_topic_" + randomAlphanumeric(3); + createTestTopic(sourceTopic, 1); + + // create producer and send one message + String value = randomAlphabetic(5); + Map properties = new HashMap<>(); + properties.put("key1", "value1"); + properties.put("key2", "value2"); + try { + Producer producer = + pulsar.operator().createProducer(sourceTopic, Schema.STRING); + producer.newMessage().value(value).properties(properties).send(); + } catch (PulsarClientException e) { + sneakyThrow(e); + } + + String sourceTableName = randomAlphabetic(5); + final String createSourceTable = + String.format( + "create table %s (\n" + + " name STRING\n," + + " `properties` MAP METADATA\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'format' = '%s'\n" + + ")", + sourceTableName, + PulsarTableFactory.IDENTIFIER, + sourceTopic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + RAW_FORMAT); + + tableEnv.executeSql(createSourceTable); + final List result = + collectRows( + tableEnv.sqlQuery(String.format("SELECT * FROM %s", sourceTableName)), 1); + assertThat(result) + .containsExactlyInAnyOrder( + Row.of(value, map(entry("key1", "value1"), entry("key2", "value2")))); + } + + // TODO split this into two tests + @ParameterizedTest + @MethodSource("provideSchemaData") + void readAndSelectIntoTableUsingSimpleSchema( + String format, Schema schema, T value, String flinkTableDataType) throws Exception { + final String sourceTopic = "source_topic_" + randomAlphanumeric(3); + createTestTopic(sourceTopic, 1); + pulsar.operator().sendMessage(sourceTopic, schema, value); + + String sourceTableName = randomAlphabetic(5); + final String createSourceTable = + String.format( + "create table %s (\n" + + " `field_1` %s\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'format' = '%s'\n" + + ")", + sourceTableName, + flinkTableDataType, + PulsarTableFactory.IDENTIFIER, + sourceTopic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + format); + + tableEnv.executeSql(createSourceTable); + final List result = + collectRows( + tableEnv.sqlQuery(String.format("SELECT * FROM %s", sourceTableName)), 1); + assertThat(result).containsExactlyInAnyOrder(Row.of(value)); + + // insert into ... select from + + final String sinkTopic = "sink_topic_" + randomAlphanumeric(3); + createTestTopic(sinkTopic, 1); + + String sinkTableName = randomAlphabetic(5); + pulsar.operator().sendMessage(sourceTopic, schema, value); + final String createSinkTable = + String.format( + "create table %s (\n" + + " `field_1` %s\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'format' = '%s'\n" + + ")", + sinkTableName, + flinkTableDataType, + PulsarTableFactory.IDENTIFIER, + sinkTopic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + format); + + tableEnv.executeSql(createSinkTable); + tableEnv.executeSql( + String.format("INSERT INTO %s SELECT * FROM %s", sinkTableName, sourceTableName)); + Message sinkResult = pulsar.operator().receiveMessage(sinkTopic, schema); + assertThat(sinkResult.getValue()).isEqualTo(value); + } + + @ParameterizedTest + @MethodSource("provideAvroBasedSchemaData") + @Disabled("flink-128") + void sendMessageToTopicAndReadUsingAvroBasedSchema( + String format, Schema schema, TestingUser value) throws Exception { + final String sourceTopic = "source_topic_" + randomAlphanumeric(3); + createTestTopic(sourceTopic, 1); + pulsar.operator().sendMessage(sourceTopic, schema, value); + + String sourceTableName = randomAlphabetic(5); + final String createSourceTable = + String.format( + "create table %s (\n" + + " name STRING,\n" + + " age INT\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'format' = '%s'\n" + + ")", + sourceTableName, + PulsarTableFactory.IDENTIFIER, + sourceTopic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + format); + + tableEnv.executeSql(createSourceTable); + final List result = + collectRows( + tableEnv.sqlQuery(String.format("SELECT * FROM %s", sourceTableName)), 1); + assertThat(result).containsExactlyInAnyOrder(Row.of(value.getName(), value.getAge())); + } + + @ParameterizedTest + @MethodSource("provideAvroBasedSchemaData") + void selectIntoTableUsingAvroBasedSchema( + String format, Schema schema, TestingUser value) throws Exception { + final String sourceTopic = "source_topic_" + randomAlphanumeric(3); + createTestTopic(sourceTopic, 1); + pulsar.operator().sendMessage(sourceTopic, schema, value); + + String sourceTableName = randomAlphabetic(5); + final String createSourceTable = + String.format( + "create table %s (\n" + + " age INT,\n" + + " name STRING\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'format' = '%s'\n" + + ")", + sourceTableName, + PulsarTableFactory.IDENTIFIER, + sourceTopic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + format); + + tableEnv.executeSql(createSourceTable); + + final String sinkTopic = "sink_topic_" + randomAlphanumeric(3); + createTestTopic(sinkTopic, 1); + String sinkTableName = randomAlphabetic(5); + final String createSinkTable = + String.format( + "create table %s (\n" + + " age INT,\n" + + " name STRING\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'format' = '%s'\n" + + ")", + sinkTableName, + PulsarTableFactory.IDENTIFIER, + sinkTopic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + format); + + tableEnv.executeSql(createSinkTable); + tableEnv.executeSql( + String.format("INSERT INTO %s SELECT * FROM %s", sinkTableName, sourceTableName)); + Message sinkResult = pulsar.operator().receiveMessage(sinkTopic, schema); + assertThat(sinkResult.getValue()).isEqualTo(value); + } + + @Test + void sendMessageToTopicAndReadUntilBoundedStopCursor() throws Exception { + final String sourceTopic = "source_topic_" + randomAlphanumeric(3); + createTestTopic(sourceTopic, 1); + pulsar.operator().sendMessage(sourceTopic, Schema.STRING, randomAlphabetic(5)); + + String sourceTableName = randomAlphabetic(5); + final String createSourceTable = + String.format( + "create table %s (\n" + + " name STRING\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'pulsar.source.partitionDiscoveryIntervalMs' = '-1',\n" + + " 'source.stop.at-message-id' = 'latest',\n" + + " 'format' = '%s'\n" + + ")", + sourceTableName, + PulsarTableFactory.IDENTIFIER, + sourceTopic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + RAW_FORMAT); + + tableEnv.executeSql(createSourceTable); + JobClient jobClient = + tableEnv.sqlQuery(String.format("SELECT * FROM %s", sourceTableName)) + .execute() + .getJobClient() + .get(); + assertThatNoException() + .isThrownBy( + () -> { + JobExecutionResult result = + jobClient.getJobExecutionResult().get(1, TimeUnit.MINUTES); + }); + } + + @Test + void sendMessageToTopicAndReadUntilBoundedStopCursorButHasPartitionDiscovery() + throws Exception { + final String sourceTopic = "source_topic_" + randomAlphanumeric(3); + createTestTopic(sourceTopic, 1); + pulsar.operator().sendMessage(sourceTopic, Schema.STRING, randomAlphabetic(5)); + + String sourceTableName = randomAlphabetic(5); + final String createSourceTable = + String.format( + "create table %s (\n" + + " name STRING\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'source.stop.at-message-id' = 'latest',\n" + + " 'format' = '%s'\n" + + ")", + sourceTableName, + PulsarTableFactory.IDENTIFIER, + sourceTopic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + RAW_FORMAT); + + tableEnv.executeSql(createSourceTable); + JobClient jobClient = + tableEnv.sqlQuery(String.format("SELECT * FROM %s", sourceTableName)) + .execute() + .getJobClient() + .get(); + assertThatExceptionOfType(TimeoutException.class) + .isThrownBy( + () -> { + JobExecutionResult result = + jobClient.getJobExecutionResult().get(1, TimeUnit.MINUTES); + }); + } + + private static final class TestingSinkFunction implements SinkFunction { + + private static final long serialVersionUID = 455430015321124493L; + private static List rows = new ArrayList<>(); + + private final int expectedSize; + + private TestingSinkFunction(int expectedSize) { + this.expectedSize = expectedSize; + rows.clear(); + } + + @Override + public void invoke(Row value, Context context) { + rows.add(value.toString()); + if (rows.size() >= expectedSize) { + // job finish + throw new SuccessException(); + } + } + } + + private static boolean isCausedByJobFinished(Throwable e) { + if (e instanceof SuccessException) { + return true; + } else if (e.getCause() != null) { + return isCausedByJobFinished(e.getCause()); + } else { + return false; + } + } + + private static Stream provideSchemaData() { + return Stream.of( + Arguments.of(RAW_FORMAT, Schema.INT8, (byte) 0xa, DataTypes.TINYINT().toString()), + Arguments.of( + RAW_FORMAT, Schema.INT16, Short.MAX_VALUE, DataTypes.SMALLINT().toString()), + Arguments.of( + RAW_FORMAT, Schema.INT32, Integer.MAX_VALUE, DataTypes.INT().toString()), + Arguments.of( + RAW_FORMAT, Schema.INT64, Long.MAX_VALUE, DataTypes.BIGINT().toString()), + Arguments.of( + RAW_FORMAT, Schema.FLOAT, Float.MAX_VALUE, DataTypes.FLOAT().toString()), + Arguments.of( + RAW_FORMAT, Schema.DOUBLE, Double.MAX_VALUE, DataTypes.DOUBLE().toString()), + Arguments.of(RAW_FORMAT, Schema.BOOL, Boolean.TRUE, DataTypes.BOOLEAN().toString()), + Arguments.of(RAW_FORMAT, Schema.BYTES, new byte[1], DataTypes.BYTES().toString()), + Arguments.of( + RAW_FORMAT, + Schema.STRING, + "this is a string", + DataTypes.STRING().toString())); + } + + private static Stream provideAvroBasedSchemaData() { + return Stream.of( + Arguments.of(JSON_FORMAT, Schema.JSON(TestingUser.class), createRandomUser()), + Arguments.of(AVRO_FORMAT, Schema.AVRO(TestingUser.class), createRandomUser())); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java new file mode 100644 index 0000000000000..53ce6bd02a2a5 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java @@ -0,0 +1,416 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.MessageIdStartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.TimestampStartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.stop.PublishTimestampStopCursor; +import org.apache.flink.connector.pulsar.table.testutils.MockTopicRouter; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.utils.FactoryMocks; +import org.apache.flink.table.types.DataType; + +import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.impl.MessageIdImpl; +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_STATS_INTERVAL_SECONDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.createKeyFormatProjection; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.createValueFormatProjection; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getMessageDelayMillis; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getPulsarProperties; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getStartCursor; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getStopCursor; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getSubscriptionType; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicListFromOptions; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicRouter; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicRoutingMode; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.parseAfterMessageIdStopCursor; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.parseAtMessageIdStopCursor; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.parseMessageIdStartCursor; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.parseMessageIdString; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_CUSTOM_TOPIC_ROUTER; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_MESSAGE_DELAY_INTERVAL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.flink.table.api.DataTypes.FIELD; +import static org.apache.flink.table.api.DataTypes.INT; +import static org.apache.flink.table.api.DataTypes.ROW; +import static org.apache.flink.table.api.DataTypes.STRING; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; +import static org.assertj.core.api.Assertions.assertThatNoException; + +/** + * Unit test for {@link PulsarTableOptionUtils}. Tests each method and different inputs. Some tests + * have overlapping semantics with {@link PulsarTableOptionsTest} and {@link + * PulsarTableValidationUtilsTest}, but they cover different aspects of the validation, so all of + * them should be kept. + */ +public class PulsarTableOptionUtilsTest { + // -------------------------------------------------------------------------------------------- + // Format and Projection Test + // -------------------------------------------------------------------------------------------- + @Test + void formatProjection() { + final DataType dataType = + DataTypes.ROW( + FIELD("id", INT()), + FIELD("name", STRING()), + FIELD("age", INT()), + FIELD("address", STRING())); + + final Map options = createTestOptions(); + options.put("key.fields", "address; name"); + + final Configuration config = Configuration.fromMap(options); + + assertThat(createKeyFormatProjection(config, dataType)).containsExactly(3, 1); + assertThat(createValueFormatProjection(config, dataType)).containsExactly(0, 2); + } + + @Test + void invalidKeyFormatFieldProjection() { + final DataType dataType = ROW(FIELD("id", INT()), FIELD("name", STRING())); + final Map options = createTestOptions(); + options.put("key.fields", "non_existing"); + + final Configuration config = Configuration.fromMap(options); + + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> createKeyFormatProjection(config, dataType)) + .withMessage( + String.format( + "Could not find the field '%s' in the table schema for usage in the key format. " + + "A key field must be a regular, physical column. " + + "The following columns can be selected in the '%s' option: [id, name]", + "non_existing", KEY_FIELDS.key())); + } + + static Map createTestOptions() { + final Map options = new HashMap<>(); + options.put("key.format", "test-format"); + options.put("key.test-format.delimiter", ","); + options.put("value.format", "test-format"); + options.put("value.test-format.delimiter", "|"); + options.put("value.test-format.fail-on-missing", "true"); + return options; + } + + // -------------------------------------------------------------------------------------------- + // Table Source Option Utils Test + // -------------------------------------------------------------------------------------------- + + @Test + void topicsList() { + final Map options = createDefaultOptions(); + options.put(TOPICS.key(), ";"); + List topicList = getTopicListFromOptions(Configuration.fromMap(options)); + assertThat(topicList).isEmpty(); + + options.put(TOPICS.key(), "topic1;"); + topicList = getTopicListFromOptions(Configuration.fromMap(options)); + assertThat(topicList).hasSize(1); + + options.put(TOPICS.key(), "topic1;topic2"); + topicList = getTopicListFromOptions(Configuration.fromMap(options)); + assertThat(topicList).hasSize(2); + + options.put(TOPICS.key(), ""); + topicList = getTopicListFromOptions(Configuration.fromMap(options)); + assertThat(topicList).isEmpty(); + } + + @Test + void pulsarProperties() { + final Map options = createDefaultOptions(); + options.put(PULSAR_STATS_INTERVAL_SECONDS.key(), "30"); + Properties properties = getPulsarProperties(Configuration.fromMap(options)); + assertThat(properties.getProperty(PULSAR_STATS_INTERVAL_SECONDS.key())).isEqualTo("30"); + } + + @Test + void startCursor() { + // TDOO Use isEqualTo() to assert; need equals() method + final Map options = createDefaultOptions(); + options.put(SOURCE_START_FROM_MESSAGE_ID.key(), "earliest"); + StartCursor startCursor = getStartCursor(Configuration.fromMap(options)); + assertThat(startCursor).isInstanceOf(MessageIdStartCursor.class); + + options.put(SOURCE_START_FROM_MESSAGE_ID.key(), "latest"); + startCursor = getStartCursor(Configuration.fromMap(options)); + assertThat(startCursor).isInstanceOf(MessageIdStartCursor.class); + + options.put(SOURCE_START_FROM_MESSAGE_ID.key(), "0:0:-1"); + startCursor = getStartCursor(Configuration.fromMap(options)); + assertThat(startCursor).isInstanceOf(MessageIdStartCursor.class); + + options.put(SOURCE_START_FROM_MESSAGE_ID.key(), "other"); + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> getStartCursor(Configuration.fromMap(options))) + .withMessage("MessageId format must be ledgerId:entryId:partitionId."); + + options.remove(SOURCE_START_FROM_MESSAGE_ID.key()); + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "123545"); + startCursor = getStartCursor(Configuration.fromMap(options)); + assertThat(startCursor).isInstanceOf(TimestampStartCursor.class); + + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "123545L"); + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> getStartCursor(Configuration.fromMap(options))) + .withMessage( + "Could not parse value '123545L' for key 'source.start.publish-time'."); + } + + @Test + void subscriptionType() { + final Map options = createDefaultOptions(); + options.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Shared"); + SubscriptionType subscriptionType = getSubscriptionType(Configuration.fromMap(options)); + assertThat(subscriptionType).isEqualTo(SubscriptionType.Shared); + + options.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Exclusive"); + subscriptionType = getSubscriptionType(Configuration.fromMap(options)); + assertThat(subscriptionType).isEqualTo(SubscriptionType.Exclusive); + } + + @Test + void canParseMessageIdEarliestOrLatestStartCursor() { + String earliest = "earliest"; + StartCursor startCursor = parseMessageIdStartCursor(earliest); + assertThat(startCursor).isEqualTo(StartCursor.earliest()); + + String latest = "latest"; + startCursor = parseMessageIdStartCursor(latest); + assertThat(startCursor).isEqualTo(StartCursor.latest()); + + String precise = "0:0:100"; + startCursor = parseMessageIdStartCursor(precise); + assertThat(startCursor).isEqualTo(StartCursor.fromMessageId(new MessageIdImpl(0, 0, 100))); + } + + @Test + void publishTimeStartCursor() { + final Map options = createDefaultOptions(); + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "12345"); + StartCursor startCursor = getStartCursor(Configuration.fromMap(options)); + assertThat(startCursor).isInstanceOf(TimestampStartCursor.class); + + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "12345L"); + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> getStartCursor(Configuration.fromMap(options))) + .withMessage("Could not parse value '12345L' for key 'source.start.publish-time'."); + } + + @Test + void canParseMessageIdNeverOrLatestStopCursor() { + String never = "never"; + StopCursor stopCursor = parseAtMessageIdStopCursor(never); + assertThat(stopCursor).isEqualTo(StopCursor.never()); + + String latest = "latest"; + stopCursor = parseAtMessageIdStopCursor(latest); + assertThat(stopCursor).isEqualTo(StopCursor.latest()); + + String precise = "0:0:100"; + stopCursor = parseAtMessageIdStopCursor(precise); + assertThat(stopCursor).isEqualTo(StopCursor.atMessageId(new MessageIdImpl(0, 0, 100))); + + stopCursor = parseAfterMessageIdStopCursor(precise); + assertThat(stopCursor).isEqualTo(StopCursor.afterMessageId(new MessageIdImpl(0, 0, 100))); + } + + @Test + void publishTimeStopCursor() { + final Map options = createDefaultOptions(); + options.put(SOURCE_STOP_AT_PUBLISH_TIME.key(), "12345"); + StopCursor stopCursor = getStopCursor(Configuration.fromMap(options)); + assertThat(stopCursor).isInstanceOf(PublishTimestampStopCursor.class); + + options.put(SOURCE_STOP_AT_PUBLISH_TIME.key(), "12345L"); + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> getStopCursor(Configuration.fromMap(options))) + .withMessage( + "Could not parse value '12345L' for key 'source.stop.at-publish-time'."); + } + + @Test + void canParseMessageIdUsingMessageIdImpl() { + final String invalidFormatMessage = + "MessageId format must be ledgerId:entryId:partitionId."; + final String invalidNumberMessage = + "MessageId format must be ledgerId:entryId:partitionId. Each id should be able to parsed to long type."; + String precise = "0:0:100"; + assertThatNoException().isThrownBy(() -> parseMessageIdString(precise)); + + String empty = ""; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdString(empty)) + .withMessage(invalidFormatMessage); + + String noSemicolon = "0"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdString(noSemicolon)) + .withMessage(invalidFormatMessage); + + String oneSemiColon = "0:"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdString(oneSemiColon)) + .withMessage(invalidFormatMessage); + + String oneSemiColonComplete = "0:0"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdString(oneSemiColonComplete)) + .withMessage(invalidFormatMessage); + + String twoSemiColon = "0:0:"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdString(twoSemiColon)) + .withMessage(invalidNumberMessage); + + String twoSemiColonComplete = "0:0:0"; + assertThatNoException().isThrownBy(() -> parseMessageIdString(twoSemiColonComplete)); + + String threeSemicolon = "0:0:0:"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdString(threeSemicolon)) + .withMessage(invalidNumberMessage); + + String threeSemicolonComplete = "0:0:0:0"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdString(threeSemicolonComplete)) + .withMessage(invalidNumberMessage); + + String invalidNumber = "0:0:adf"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdString(invalidNumber)) + .withMessage(invalidNumberMessage); + } + + // -------------------------------------------------------------------------------------------- + // Table Sink Option Utils Test + // -------------------------------------------------------------------------------------------- + + @Test + void topicRouter() { + final Map options = createDefaultOptions(); + options.put( + SINK_CUSTOM_TOPIC_ROUTER.key(), + "org.apache.flink.connector.pulsar.table.testutils.MockTopicRouter"); + TopicRouter topicRouter = + getTopicRouter(Configuration.fromMap(options), FactoryMocks.class.getClassLoader()); + assertThat(topicRouter).isInstanceOf(MockTopicRouter.class); + + options.put( + SINK_CUSTOM_TOPIC_ROUTER.key(), + "org.apache.flink.connector.pulsar.table.PulsarTableOptionsTest"); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy( + () -> + getTopicRouter( + Configuration.fromMap(options), + FactoryMocks.class.getClassLoader())) + .withMessage( + String.format( + "Sink TopicRouter class '%s' should extend from the required class %s", + PulsarTableOptionsTest.class.getName(), + TopicRouter.class.getName())); + + options.put( + SINK_CUSTOM_TOPIC_ROUTER.key(), + "org.apache.flink.connector.pulsar.table.testutils.NonExistMockTopicRouter"); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy( + () -> + getTopicRouter( + Configuration.fromMap(options), + FactoryMocks.class.getClassLoader())) + .withMessage( + String.format( + "Could not find and instantiate TopicRouter class '%s'", + "org.apache.flink.connector.pulsar.table.testutils.NonExistMockTopicRouter")); + } + + @Test + void topicRoutingMode() { + final Map options = createDefaultOptions(); + options.put(SINK_TOPIC_ROUTING_MODE.key(), "round-robin"); + TopicRoutingMode topicRoutingMode = getTopicRoutingMode(Configuration.fromMap(options)); + assertThat(topicRoutingMode).isEqualTo(TopicRoutingMode.ROUND_ROBIN); + + options.put(SINK_TOPIC_ROUTING_MODE.key(), "message-key-hash"); + topicRoutingMode = getTopicRoutingMode(Configuration.fromMap(options)); + assertThat(topicRoutingMode).isEqualTo(TopicRoutingMode.MESSAGE_KEY_HASH); + } + + @Test + void messageDelayMillis() { + final Map options = createDefaultOptions(); + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "10 s"); + long messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofSeconds(10).toMillis()); + + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "10s"); + messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofSeconds(10).toMillis()); + + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "1000ms"); + messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofMillis(1000).toMillis()); + + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "1 d"); + messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofDays(1).toMillis()); + + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "1 H"); + messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofHours(1).toMillis()); + + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "1 min"); + messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofMinutes(1).toMillis()); + + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "1m"); + messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofMinutes(1).toMillis()); + } + + private Map createDefaultOptions() { + Map optionMap = new HashMap<>(); + return optionMap; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionsTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionsTest.java new file mode 100644 index 0000000000000..f48f070364a00 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionsTest.java @@ -0,0 +1,514 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table; + +import org.apache.flink.connector.pulsar.table.testutils.MockPulsarAuthentication; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.factories.FactoryUtil; + +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_AUTH_PARAM_MAP; +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_AUTH_PLUGIN_CLASS_NAME; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_INITIAL_POSITION; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_CUSTOM_TOPIC_ROUTER; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_MESSAGE_DELAY_INTERVAL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.flink.table.factories.TestDynamicTableFactory.VALUE_FORMAT; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; +import static org.assertj.core.api.Assertions.assertThatNoException; + +/** + * Test config options for Pulsar SQL connector. This test aims to verify legal combination of + * config options will be accepted and do not cause runtime exceptions (but cannot guarantee they + * are taking effect), and illegal combinations of config options will be rejected early. + */ +public class PulsarTableOptionsTest extends PulsarTableTestBase { + @Test + void noTopicsSpecified() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithFormat(); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException( + topicName, + new ValidationException( + "One or more required options are missing.\n\n" + + "Missing required options are:\n\n" + + "topics")); + } + + @Test + void invalidEmptyTopics() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithFormat(); + testConfigs.put(TOPICS.key(), ""); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException( + topicName, new ValidationException("The topics list should not be empty.")); + } + + @Test + void topicsWithSemicolon() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithFormat(); + testConfigs.put(TOPICS.key(), topicName + ";"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void invalidTopicName() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithFormat(); + String invalidTopicName = "persistent://tenant/no-topic"; + testConfigs.put(TOPICS.key(), invalidTopicName); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException( + topicName, + new ValidationException( + String.format( + "The topics name %s is not a valid topic name.", + invalidTopicName))); + } + + @Test + void topicsList() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithFormat(); + testConfigs.put( + TOPICS.key(), + topicNameWithPartition(topicName, 0) + ";" + topicNameWithPartition(topicName, 1)); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void usingFormat() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + testConfigs.put(FactoryUtil.FORMAT.key(), "json"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void usingValueFormat() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + testConfigs.put(VALUE_FORMAT.key(), "json"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void usingValueFormatAndFormatOptions() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + testConfigs.put(VALUE_FORMAT.key(), "json"); + testConfigs.put("value.json.fail-on-missing-field", "false"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void subscriptionType() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + testConfigs.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Exclusive"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void invalidUnsupportedSubscriptionType() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + testConfigs.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Key_Shared"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectException( + topicName, + new ValidationException( + "Only Exclusive and Shared SubscriptionType is supported. ")); + } + + @Test + void invalidNonExistSubscriptionType() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + testConfigs.put(SOURCE_SUBSCRIPTION_TYPE.key(), "random-subscription-type"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException( + topicName, + new ValidationException("Invalid value for option 'source.subscription-type'.")); + } + + @Test + void messageIdStartCursorEarliest() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SOURCE_START_FROM_MESSAGE_ID.key(), "earliest"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void messageIdStartCursorLatest() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SOURCE_START_FROM_MESSAGE_ID.key(), "latest"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void messageIdStartCursorExact() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SOURCE_START_FROM_MESSAGE_ID.key(), "0:0:-1"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void invalidMessageIdStartCursorEmptyId() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SOURCE_START_FROM_MESSAGE_ID.key(), "0:0:"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectException( + topicName, + new IllegalArgumentException( + "MessageId format must be ledgerId:entryId:partitionId. " + + "Each id should be able to parsed to long type.")); + } + + @Test + void invalidMessageIdStartCursorIncomplete() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SOURCE_START_FROM_MESSAGE_ID.key(), "0:0"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectException( + topicName, + new IllegalArgumentException( + "MessageId format must be ledgerId:entryId:partitionId.")); + } + + @Test + void timestampStartCursor() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "233010230"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void messageIdStopCursorNever() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SOURCE_STOP_AT_MESSAGE_ID.key(), "never"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void messageIdStopCursorLatest() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SOURCE_STOP_AT_MESSAGE_ID.key(), "latest"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void messageIdStopCursorExact() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SOURCE_STOP_AT_MESSAGE_ID.key(), "0:0:-1"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void timestampStopCursor() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SOURCE_STOP_AT_PUBLISH_TIME.key(), "233010230"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void topicRoutingMode() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SINK_TOPIC_ROUTING_MODE.key(), "message-key-hash"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void invalidTopicRouter() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + String invalidClassName = "invalid class name"; + testConfigs.put(SINK_CUSTOM_TOPIC_ROUTER.key(), invalidClassName); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException( + topicName, + new ValidationException( + String.format( + "Could not find and instantiate TopicRouter class '%s'", + invalidClassName))); + } + + @Test + void messageDelay() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "10s"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void invalidMessageDelay() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "invalid-duration"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException( + topicName, + new ValidationException("Invalid value for option 'sink.message-delay-interval'.")); + } + + // -------------------------------------------------------------------------------------------- + // PulsarSourceOptions Test + // -------------------------------------------------------------------------------------------- + @Test + void subscriptionInitialPosition() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(PULSAR_SUBSCRIPTION_INITIAL_POSITION.key(), "Earliest"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + // -------------------------------------------------------------------------------------------- + // PulsarOptions, PulsarSourceOptions, PulsarSinkOptions Test + // -------------------------------------------------------------------------------------------- + + @Test + void pulsarOptionsAuthParamMap() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(PULSAR_AUTH_PARAM_MAP.key(), "key1:value1,key2:value2"); + testConfigs.put( + PULSAR_AUTH_PLUGIN_CLASS_NAME.key(), MockPulsarAuthentication.class.getName()); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + // -------------------------------------------------------------------------------------------- + // requiredOptions(), optionalOptions() Test + // -------------------------------------------------------------------------------------------- + + @Test + void unusedConfigOptions() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put("random_config", "random_value"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException(topicName, ValidationException.class); + } + + // -------------------------------------------------------------------------------------------- + // Utils methods + // -------------------------------------------------------------------------------------------- + + private String createTestConfig(Map configMap) { + StringBuilder sb = new StringBuilder(); + for (Map.Entry entry : configMap.entrySet()) { + sb.append(String.format(" '%s' = '%s' ,\n", entry.getKey(), entry.getValue())); + } + return sb.toString(); + } + + private void runSql(String topicName, String testConfigString) { + createTestTopic(topicName, 2); + final String createTable = + String.format( + "CREATE TABLE %s (\n" + + " `physical_1` STRING,\n" + + " `physical_2` INT,\n" + + " `physical_3` BOOLEAN\n" + + ") WITH (\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " %s\n" + + " 'connector' = 'pulsar'" + + ")", + topicName, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + testConfigString); + tableEnv.executeSql(createTable); + } + + private void runSinkAndExpectSucceed(String topicName) { + String initialValues = + String.format( + "INSERT INTO %s\n" + + "VALUES\n" + + " ('data 1', 1, TRUE),\n" + + " ('data 2', 2, FALSE),\n" + + " ('data 3', 3, TRUE)", + topicName); + assertThatNoException().isThrownBy(() -> tableEnv.executeSql(initialValues).await()); + } + + private void runSinkAndExpectException( + String topicName, final Class exceptionType) { + String initialValues = + String.format( + "INSERT INTO %s\n" + + "VALUES\n" + + " ('data 1', 1, TRUE),\n" + + " ('data 2', 2, FALSE),\n" + + " ('data 3', 3, TRUE)", + topicName); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> tableEnv.executeSql(initialValues).await()); + } + + private void runSinkAndExpectException(String topicName, Throwable cause) { + String initialValues = + String.format( + "INSERT INTO %s\n" + + "VALUES\n" + + " ('data 1', 1, TRUE),\n" + + " ('data 2', 2, FALSE),\n" + + " ('data 3', 3, TRUE)", + topicName); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> tableEnv.executeSql(initialValues).await()) + .withCause(cause); + } + + private void runSourceAndExpectSucceed(String topicName) { + assertThatNoException() + .isThrownBy(() -> tableEnv.sqlQuery(String.format("SELECT * FROM %s", topicName))); + } + + private void runSourceAndExpectException(String topicName, Throwable cause) { + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> tableEnv.sqlQuery(String.format("SELECT * FROM %s", topicName))) + .withCause(cause); + } + + private String randomTopicName() { + final String testTopicPrefix = "test_config_topic"; + return testTopicPrefix + randomAlphabetic(5); + } + + private Map testConfigWithTopicAndFormat(String tableName) { + Map testConfigs = new HashMap<>(); + testConfigs.put(TOPICS.key(), tableName); + testConfigs.put(FactoryUtil.FORMAT.key(), "json"); + return testConfigs; + } + + private Map testConfigWithFormat() { + Map testConfigs = new HashMap<>(); + testConfigs.put(FactoryUtil.FORMAT.key(), "json"); + return testConfigs; + } + + private Map testConfigWithTopic(String tableName) { + Map testConfigs = new HashMap<>(); + testConfigs.put(TOPICS.key(), tableName); + return testConfigs; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java new file mode 100644 index 0000000000000..32ce79ffccff1 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table; + +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; +import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntime; +import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment; +import org.apache.flink.connector.testframe.junit.annotations.TestEnv; +import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem; +import org.apache.flink.connector.testframe.junit.annotations.TestSemantics; +import org.apache.flink.streaming.api.CheckpointingMode; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestInstance; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Base class for Pulsar table integration test. */ +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +public abstract class PulsarTableTestBase { + private static final Logger LOG = LoggerFactory.getLogger(PulsarTableTestBase.class); + + @TestEnv MiniClusterTestEnvironment flink = new MiniClusterTestEnvironment(); + + // Defines pulsar running environment + @TestExternalSystem + protected PulsarTestEnvironment pulsar = new PulsarTestEnvironment(runtime()); + + @TestSemantics + protected CheckpointingMode[] semantics = + new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE}; + + protected StreamExecutionEnvironment env; + + protected StreamTableEnvironment tableEnv; + + protected PulsarRuntime runtime() { + return PulsarRuntime.container(); + } + + @BeforeAll + public void beforeAll() { + pulsar.startUp(); + // run env + env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + env.getConfig().setRestartStrategy(RestartStrategies.noRestart()); + tableEnv = StreamTableEnvironment.create(env); + tableEnv.getConfig() + .getConfiguration() + .setString("table.dynamic-table-options.enabled", "true"); + } + + public void createTestTopic(String topic, int numPartitions) { + pulsar.operator().createTopic(topic, numPartitions); + } + + @AfterAll + public void afterAll() { + pulsar.tearDown(); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtilsTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtilsTest.java new file mode 100644 index 0000000000000..27c1133068aa5 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtilsTest.java @@ -0,0 +1,193 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.ValidationException; + +import org.apache.pulsar.client.api.SubscriptionType; +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FORMAT; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_CUSTOM_TOPIC_ROUTER; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateKeyFormatConfigs; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateSinkRoutingConfigs; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateStartCursorConfigs; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateSubscriptionTypeConfigs; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateTopicsConfigs; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; +import static org.assertj.core.api.Assertions.assertThatNoException; + +/** Unit test for {@link PulsarTableValidationUtils}. */ +public class PulsarTableValidationUtilsTest extends PulsarTableTestBase { + @Test + void topicsConfigs() { + final Map options = createDefaultOptions(); + options.put(TOPICS.key(), ""); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateTopicsConfigs(Configuration.fromMap(options))) + .withMessage("The topics list should not be empty."); + + String invalidTopicName = "persistent://tenant/topic"; + String validTopicName = "valid-topic"; + + options.put(TOPICS.key(), invalidTopicName); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateTopicsConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "The topics name %s is not a valid topic name.", invalidTopicName)); + + options.put(TOPICS.key(), validTopicName + ";" + invalidTopicName); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateTopicsConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "The topics name %s is not a valid topic name.", invalidTopicName)); + + options.put(TOPICS.key(), validTopicName + ";" + validTopicName); + assertThatNoException() + .isThrownBy(() -> validateTopicsConfigs(Configuration.fromMap(options))); + + options.put(TOPICS.key(), validTopicName + ";"); + assertThatNoException() + .isThrownBy(() -> validateTopicsConfigs(Configuration.fromMap(options))); + } + + @Test + void startCursorConfigs() { + final Map options = createDefaultOptions(); + options.put(SOURCE_START_FROM_MESSAGE_ID.key(), "latest"); + assertThatNoException() + .isThrownBy(() -> validateStartCursorConfigs(Configuration.fromMap(options))); + + options.remove(SOURCE_START_FROM_MESSAGE_ID.key()); + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "2345123234"); + assertThatNoException() + .isThrownBy(() -> validateStartCursorConfigs(Configuration.fromMap(options))); + + options.put(SOURCE_START_FROM_MESSAGE_ID.key(), "latest"); + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "2345123234"); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateStartCursorConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "Only one of %s and %s can be specified. Detected both of them", + SOURCE_START_FROM_MESSAGE_ID, SOURCE_START_FROM_PUBLISH_TIME)); + } + + @Test + void subscriptionTypeConfigs() { + final Map options = createDefaultOptions(); + + options.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Exclusive"); + assertThatNoException() + .isThrownBy(() -> validateSubscriptionTypeConfigs(Configuration.fromMap(options))); + + options.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Shared"); + assertThatNoException() + .isThrownBy(() -> validateSubscriptionTypeConfigs(Configuration.fromMap(options))); + + options.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Key_Shared"); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateSubscriptionTypeConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "Only %s and %s SubscriptionType is supported. ", + SubscriptionType.Exclusive, SubscriptionType.Shared)); + + options.put(SOURCE_SUBSCRIPTION_TYPE.key(), "invalid-subscription"); + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> validateSubscriptionTypeConfigs(Configuration.fromMap(options))) + .withMessage( + "Could not parse value 'invalid-subscription' for key 'source.subscription-type'."); + } + + @Test + void sinkRoutingConfigs() { + final Map options = createDefaultOptions(); + options.put(SINK_TOPIC_ROUTING_MODE.key(), "round-robin"); + assertThatNoException() + .isThrownBy(() -> validateSinkRoutingConfigs(Configuration.fromMap(options))); + + // validation does not try to create the class + options.remove(SINK_TOPIC_ROUTING_MODE.key()); + options.put(SINK_CUSTOM_TOPIC_ROUTER.key(), "invalid-class-name"); + assertThatNoException() + .isThrownBy(() -> validateSinkRoutingConfigs(Configuration.fromMap(options))); + + options.put(SINK_TOPIC_ROUTING_MODE.key(), "round-robin"); + options.put(SINK_CUSTOM_TOPIC_ROUTER.key(), "invalid-class-name"); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateSinkRoutingConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "Only one of %s and %s can be specified. Detected both of them", + SINK_CUSTOM_TOPIC_ROUTER, SINK_TOPIC_ROUTING_MODE)); + } + + @Test + void keyFormatConfigs() { + final Map options = createDefaultOptions(); + options.put(KEY_FIELDS.key(), ""); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateKeyFormatConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "The option '%s' can only be declared if a key format is defined using '%s'.", + KEY_FIELDS.key(), KEY_FORMAT.key())); + + options.put(KEY_FORMAT.key(), "json"); + options.remove(KEY_FIELDS.key()); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateKeyFormatConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "A key format '%s' requires the declaration of one or more of key fields using '%s'.", + KEY_FORMAT.key(), KEY_FIELDS.key())); + + options.put(KEY_FORMAT.key(), "json"); + options.put(KEY_FIELDS.key(), ""); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateKeyFormatConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "A key format '%s' requires the declaration of one or more of key fields using '%s'.", + KEY_FORMAT.key(), KEY_FIELDS.key())); + + options.put(KEY_FORMAT.key(), "json"); + options.put(KEY_FIELDS.key(), "k_field1"); + assertThatNoException() + .isThrownBy(() -> validateKeyFormatConfigs(Configuration.fromMap(options))); + } + + private Map createDefaultOptions() { + Map optionMap = new HashMap<>(); + return optionMap; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogITTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogITTest.java new file mode 100644 index 0000000000000..8579f87886580 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogITTest.java @@ -0,0 +1,928 @@ +package org.apache.flink.connector.pulsar.table.catalog; + +import org.apache.flink.connector.pulsar.common.config.PulsarConfigBuilder; +import org.apache.flink.connector.pulsar.common.config.PulsarOptions; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; +import org.apache.flink.connector.pulsar.table.PulsarTableFactory; +import org.apache.flink.connector.pulsar.table.PulsarTableOptions; +import org.apache.flink.connector.pulsar.table.PulsarTableTestBase; +import org.apache.flink.connector.pulsar.table.testutils.TestingUser; +import org.apache.flink.formats.raw.RawFormatFactory; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.GenericInMemoryCatalog; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.types.Row; + +import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables; +import org.apache.flink.shaded.guava30.com.google.common.collect.Sets; + +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.shade.org.apache.commons.lang3.RandomStringUtils; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalog.DEFAULT_DB; +import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogFactory.CATALOG_CONFIG_VALIDATOR; +import static org.apache.flink.connector.pulsar.table.testutils.PulsarTableTestUtils.collectRows; +import static org.apache.flink.connector.pulsar.table.testutils.SchemaData.INTEGER_LIST; +import static org.apache.flink.connector.pulsar.table.testutils.TestingUser.createRandomUser; +import static org.apache.flink.connector.pulsar.table.testutils.TestingUser.createUser; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; +import static org.assertj.core.api.Assertions.assertThatNoException; + +/** Unit test for {@link PulsarCatalog}. */ +public class PulsarCatalogITTest extends PulsarTableTestBase { + private static final Logger LOGGER = LoggerFactory.getLogger(PulsarCatalogITTest.class); + + private static final String AVRO_FORMAT = "avro"; + private static final String JSON_FORMAT = "json"; + + private static final String INMEMORY_CATALOG = "inmemorycatalog"; + private static final String PULSAR_CATALOG1 = "pulsarcatalog1"; + private static final String PULSAR_CATALOG2 = "pulsarcatalog2"; + + private static final String INMEMORY_DB = "mydatabase"; + private static final String PULSAR1_DB = "public/default"; + private static final String PULSAR2_DB = "tn/ns"; + + private static final String FLINK_TENANT = "__flink_catalog"; + + @BeforeAll + void before() { + registerCatalogs(tableEnv); + } + + // catalog operations + @Test + void createCatalogWithAllConfig() throws PulsarAdminException { + String catalogName = RandomStringUtils.randomAlphabetic(10); + String customTenantPath = "__flink_custom_tenant"; + String defaultDatabase = "my_db"; + tableEnv.executeSql( + String.format( + "CREATE CATALOG %s WITH (" + + "'type' = 'pulsar-catalog',\n" + + "'catalog-admin-url' = '%s',\n" + + "'catalog-service-url' = '%s',\n" + + "'catalog-tenant' = '%s',\n" + + "'default-database' = '%s'\n" + + ")", + catalogName, + pulsar.operator().adminUrl(), + pulsar.operator().serviceUrl(), + customTenantPath, + defaultDatabase)); + Optional catalogOptional = tableEnv.getCatalog(catalogName); + assertThat(catalogOptional).isPresent(); + tableEnv.useCatalog(catalogName); + assertThat(tableEnv.getCurrentDatabase()).isEqualTo(defaultDatabase); + assertThat(pulsar.operator().admin().tenants().getTenants()).contains(customTenantPath); + } + + @Test + void createMultipleCatalog() { + tableEnv.useCatalog(INMEMORY_CATALOG); + assertThat(tableEnv.getCurrentCatalog()).isEqualTo(INMEMORY_CATALOG); + assertThat(tableEnv.getCurrentDatabase()).isEqualTo(INMEMORY_DB); + + Catalog catalog = tableEnv.getCatalog(PULSAR_CATALOG1).orElse(null); + assertThat(catalog).isNotNull(); + assertThat(catalog).isInstanceOf(PulsarCatalog.class); + + tableEnv.useCatalog(PULSAR_CATALOG1); + assertThat(tableEnv.getCurrentDatabase()).isEqualTo(PULSAR1_DB); + + catalog = tableEnv.getCatalog(PULSAR_CATALOG2).orElse(null); + assertThat(catalog).isNotNull(); + assertThat(catalog).isInstanceOf(PulsarCatalog.class); + + tableEnv.useCatalog(PULSAR_CATALOG2); + assertThat(tableEnv.getCurrentDatabase()).isEqualTo(PULSAR2_DB); + } + + // database operations + @Test + void listPulsarNativeDatabase() throws Exception { + List namespaces = Arrays.asList("tn1/ns1", "tn1/ns2"); + List topics = Arrays.asList("tp1", "tp2"); + List topicsFullName = + topics.stream().map(a -> "tn1/ns1/" + a).collect(Collectors.toList()); + List partitionedTopics = Arrays.asList("ptp1", "ptp2"); + List partitionedTopicsFullName = + partitionedTopics.stream().map(a -> "tn1/ns1/" + a).collect(Collectors.toList()); + + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + assertThat(tableEnv.getCurrentDatabase()).isEqualTo(PULSAR1_DB); + pulsar.operator() + .admin() + .tenants() + .createTenant( + "tn1", + TenantInfo.builder() + .adminRoles(Sets.newHashSet()) + .allowedClusters(Sets.newHashSet("standalone")) + .build()); + + for (String ns : namespaces) { + pulsar.operator().admin().namespaces().createNamespace(ns); + } + + for (String tp : topicsFullName) { + pulsar.operator().admin().topics().createNonPartitionedTopic(tp); + } + + for (String tp : partitionedTopicsFullName) { + pulsar.operator().admin().topics().createPartitionedTopic(tp, 5); + } + + assertThat(Sets.newHashSet(tableEnv.listDatabases())).containsAll(namespaces); + tableEnv.useDatabase("tn1/ns1"); + + Set tableSet = Sets.newHashSet(tableEnv.listTables()); + + assertThat(tableSet) + .containsExactlyInAnyOrderElementsOf(Iterables.concat(topics, partitionedTopics)); + } + + @Test + void createExplicitDatabase() { + tableEnv.useCatalog(PULSAR_CATALOG1); + String explictDatabaseName = newDatabaseName(); + assertThatNoException() + .isThrownBy( + () -> + tableEnv.executeSql( + String.format("CREATE DATABASE %s", explictDatabaseName))); + } + + @Test + void createCatalogAndExpectDefaultDatabase() + throws ExecutionException, InterruptedException, TimeoutException { + tableEnv.useCatalog(PULSAR_CATALOG1); + assertThatNoException() + .isThrownBy(() -> tableEnv.executeSql(String.format("USE %s", DEFAULT_DB))); + + assertThatNoException() + .isThrownBy(() -> tableEnv.executeSql(String.format("SHOW TABLES", DEFAULT_DB))); + + String tableName = newTopicName(); + String tableDDL = + String.format( + "CREATE TABLE %s (\n" + + " oid STRING,\n" + + " totalprice INT,\n" + + " customerid STRING\n" + + ")", + tableName); + tableEnv.executeSql(tableDDL).await(10, TimeUnit.SECONDS); + assertThat(tableEnv.listTables()).contains(tableName); + } + + @Test + void createNativeDatabaseShouldFail() { + tableEnv.useCatalog(PULSAR_CATALOG1); + String nativeDatabaseName = "tn1/ns1"; + assertThatExceptionOfType(TableException.class) + .isThrownBy( + () -> + tableEnv.executeSql( + String.format("CREATE DATABASE `%s`", nativeDatabaseName))) + .withMessageStartingWith("Could not execute CREATE DATABASE"); + } + + @Test + void dropNativeDatabaseShouldFail() { + tableEnv.useCatalog(PULSAR_CATALOG1); + assertThatExceptionOfType(TableException.class) + .isThrownBy( + () -> tableEnv.executeSql(String.format("DROP DATABASE `%s`", PULSAR1_DB))) + .withMessageStartingWith("Could not execute DROP DATABASE"); + } + + @Test + void dropExplicitDatabaseWithTablesShouldFail() { + tableEnv.useCatalog(PULSAR_CATALOG1); + String explictDatabaseName = newDatabaseName(); + String topicName = newTopicName(); + + String dbDDL = "CREATE DATABASE " + explictDatabaseName; + tableEnv.executeSql(dbDDL).print(); + tableEnv.useDatabase(explictDatabaseName); + + String createTableSql = + String.format( + "CREATE TABLE %s (\n" + + " oid STRING,\n" + + " totalprice INT,\n" + + " customerid STRING\n" + + ")", + topicName); + tableEnv.executeSql(createTableSql); + + assertThatExceptionOfType(ValidationException.class) + .isThrownBy( + () -> + tableEnv.executeSql( + String.format("DROP DATABASE %s", explictDatabaseName))) + .withMessageStartingWith("Could not execute DROP DATABASE") + .withCauseInstanceOf(DatabaseNotEmptyException.class); + } + + @Test + void dropExplicitDatabase() { + tableEnv.useCatalog(PULSAR_CATALOG1); + String explictDatabaseName = newDatabaseName(); + tableEnv.executeSql(String.format("CREATE DATABASE %s", explictDatabaseName)); + + assertThatNoException() + .isThrownBy( + () -> + tableEnv.executeSql( + String.format("DROP DATABASE %s", explictDatabaseName))); + } + + @Test + void createAndGetDetailedDatabase() throws DatabaseNotExistException { + tableEnv.useCatalog(PULSAR_CATALOG1); + String explictDatabaseName = newDatabaseName(); + tableEnv.executeSql( + String.format( + "CREATE DATABASE %s \n" + + "COMMENT 'this is a comment'\n" + + "WITH (" + + "'p1' = 'k1',\n" + + "'p2' = 'k2' \n" + + ")", + explictDatabaseName)); + tableEnv.useDatabase(explictDatabaseName); + Catalog catalog = tableEnv.getCatalog(PULSAR_CATALOG1).get(); + CatalogDatabase database = catalog.getDatabase(explictDatabaseName); + + Map expectedProperties = new HashMap<>(); + expectedProperties.put("p1", "k1"); + expectedProperties.put("p2", "k2"); + assertThat(database.getProperties()).containsAllEntriesOf(expectedProperties); + assertThat(database.getComment()).isEqualTo("this is a comment"); + } + + // table operations + @Test + void createExplicitTable() throws Exception { + String databaseName = newDatabaseName(); + String tableTopic = newTopicName(); + String tableName = TopicName.get(tableTopic).getLocalName(); + + tableEnv.useCatalog(PULSAR_CATALOG1); + + String dbDDL = "CREATE DATABASE " + databaseName; + tableEnv.executeSql(dbDDL).print(); + tableEnv.useDatabase(databaseName); + + String tableDDL = + String.format( + "CREATE TABLE %s (\n" + + " oid STRING,\n" + + " totalprice INT,\n" + + " customerid STRING\n" + + ")", + tableName); + tableEnv.executeSql(tableDDL).await(10, TimeUnit.SECONDS); + assertThat(tableEnv.listTables()).contains(tableName); + } + + @Test + void createExplicitTableAndRunSourceSink() { + tableEnv.useCatalog(PULSAR_CATALOG1); + + String databaseName = newDatabaseName(); + String dbDDL = "CREATE DATABASE " + databaseName; + tableEnv.executeSql(dbDDL).print(); + tableEnv.useDatabase(databaseName); + + String topicName = newTopicName(); + String createSql = + String.format( + "CREATE TABLE %s (\n" + + " oid STRING,\n" + + " totalprice INT,\n" + + " customerid STRING\n" + + ") with (\n" + + " 'connector' = 'pulsar',\n" + + " 'topics' = '%s'," + + " 'format' = 'avro'\n" + + ")", + topicName, topicName); + assertThatNoException().isThrownBy(() -> tableEnv.executeSql(createSql)); + } + + @Test + void createExplicitTableInNativePulsarDatabaseShouldFail() { + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + String topicName = newTopicName(); + String createSql = + String.format( + "CREATE TABLE %s (\n" + + " oid STRING,\n" + + " totalprice INT,\n" + + " customerid STRING\n" + + ") with (\n" + + " 'connector' = 'pulsar',\n" + + " 'topics' = '%s'," + + " 'format' = 'avro'\n" + + ")", + topicName, topicName); + + assertThatExceptionOfType(TableException.class) + .isThrownBy(() -> tableEnv.executeSql(createSql)) + .withMessage( + String.format( + "Could not execute CreateTable in path `%s`.`%s`.`%s`", + PULSAR_CATALOG1, PULSAR1_DB, topicName)); + } + + @Test + void dropNativeTableShouldFail() { + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + String topicName = newTopicName(); + pulsar.operator().createTopic(topicName, 1); + assertThatExceptionOfType(TableException.class) + .isThrownBy(() -> tableEnv.executeSql(String.format("DROP TABLE %s", topicName))) + .withMessage( + String.format( + "Could not execute DropTable in path `%s`.`%s`.`%s`", + PULSAR_CATALOG1, PULSAR1_DB, topicName)); + } + + @Test + void dropExplicitTable() { + tableEnv.useCatalog(PULSAR_CATALOG1); + String explictDatabaseName = newDatabaseName(); + tableEnv.executeSql(String.format("CREATE DATABASE %s", explictDatabaseName)); + tableEnv.useDatabase(explictDatabaseName); + + String topicName = newTopicName(); + String createTableSql = + String.format( + "CREATE TABLE %s (\n" + + " oid STRING,\n" + + " totalprice INT,\n" + + " customerid STRING\n" + + ")", + topicName); + tableEnv.executeSql(createTableSql); + + assertThatNoException() + .isThrownBy(() -> tableEnv.executeSql(String.format("DROP TABLE %s", topicName))); + } + + @Test + void tableExists() throws ExecutionException, InterruptedException { + tableEnv.useCatalog(PULSAR_CATALOG1); + String explictDatabaseName = newDatabaseName(); + tableEnv.executeSql(String.format("CREATE DATABASE %s", explictDatabaseName)); + tableEnv.useDatabase(explictDatabaseName); + + String topicName = newTopicName(); + String createTableSql = + String.format( + "CREATE TABLE %s (\n" + + " oid STRING,\n" + + " totalprice INT,\n" + + " customerid STRING\n" + + ")", + topicName); + tableEnv.executeSql(createTableSql).await(); + + Catalog catalog = tableEnv.getCatalog(PULSAR_CATALOG1).get(); + assertThat(catalog.tableExists(new ObjectPath(explictDatabaseName, topicName))).isTrue(); + } + + @Test + void getExplicitTable() throws TableNotExistException { + tableEnv.useCatalog(PULSAR_CATALOG1); + String explictDatabaseName = newDatabaseName(); + tableEnv.executeSql(String.format("CREATE DATABASE %s", explictDatabaseName)); + tableEnv.useDatabase(explictDatabaseName); + + String topicName = newTopicName(); + String createTableSql = + String.format( + "CREATE TABLE %s (\n" + + " oid STRING,\n" + + " totalprice INT,\n" + + " customerid STRING\n" + + ")", + topicName); + tableEnv.executeSql(createTableSql); + + Catalog catalog = tableEnv.getCatalog(PULSAR_CATALOG1).get(); + CatalogBaseTable table = catalog.getTable(new ObjectPath(explictDatabaseName, topicName)); + + Map expectedOptions = new HashMap<>(); + expectedOptions.put(PulsarTableOptions.EXPLICIT.key(), "true"); + expectedOptions.put(PulsarTableOptions.ADMIN_URL.key(), pulsar.operator().adminUrl()); + expectedOptions.put(PulsarTableOptions.SERVICE_URL.key(), pulsar.operator().serviceUrl()); + expectedOptions.put(FactoryUtil.FORMAT.key(), RawFormatFactory.IDENTIFIER); + expectedOptions.put(FactoryUtil.CONNECTOR.key(), PulsarTableFactory.IDENTIFIER); + + assertThat(table.getOptions()).containsExactlyEntriesOf(expectedOptions); + } + + @Test + void getNativeTable() throws Exception { + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + + String nativeTopicName = newTopicName(); + pulsar.operator().createTopic(nativeTopicName, 1); + + Catalog catalog = tableEnv.getCatalog(PULSAR_CATALOG1).get(); + CatalogBaseTable table = catalog.getTable(new ObjectPath(PULSAR1_DB, nativeTopicName)); + + Map expectedOptions = new HashMap<>(); + expectedOptions.put( + PulsarTableOptions.TOPICS.key(), TopicNameUtils.topicName(nativeTopicName)); + expectedOptions.put(PulsarTableOptions.ADMIN_URL.key(), pulsar.operator().adminUrl()); + expectedOptions.put(PulsarTableOptions.SERVICE_URL.key(), pulsar.operator().serviceUrl()); + expectedOptions.put(FactoryUtil.FORMAT.key(), RawFormatFactory.IDENTIFIER); + expectedOptions.put(FactoryUtil.CONNECTOR.key(), PulsarTableFactory.IDENTIFIER); + + assertThat(table.getOptions()).containsExactlyEntriesOf(expectedOptions); + } + + // runtime behaviour + + @Test + void readFromNativeTable() throws Exception { + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + String topicName = newTopicName(); + + pulsar.operator().createTopic(topicName, 1); + pulsar.operator().sendMessages(topicName, Schema.INT32, INTEGER_LIST); + + final List result = + collectRows( + tableEnv.sqlQuery(String.format("SELECT * FROM %s", topicName)), + INTEGER_LIST.size()); + assertThat(result) + .containsExactlyElementsOf( + INTEGER_LIST.stream().map(Row::of).collect(Collectors.toList())); + } + + @Test + void readFromNativeTableWithMetadata() { + // TODO this test will be implemented after useMetadata is supported; + } + + @Test + void readFromNativeTableFromEarliest() throws Exception { + String topicName = newTopicName(); + pulsar.operator().sendMessages(topicName, Schema.INT32, INTEGER_LIST); + + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.getConfig() + .getConfiguration() + .setString("table.dynamic-table-options.enabled", "true"); + tableEnv.useDatabase(PULSAR1_DB); + + final List result = + collectRows( + tableEnv.sqlQuery( + "select `value` from " + + TopicName.get(topicName).getLocalName() + + " /*+ OPTIONS('source.start.message-id'='earliest') */"), + INTEGER_LIST.size()); + assertThat(result) + .containsExactlyElementsOf( + INTEGER_LIST.stream().map(Row::of).collect(Collectors.toList())); + } + + @Test + void readFromNativeTableWithProtobufNativeSchema() { + // TODO implement after protobuf native schema support + } + + // TODO we didn't create the topic, how can we send to it ? + @ParameterizedTest + @MethodSource("provideAvroBasedSchemaData") + void readFromNativeTableWithAvroBasedSchema(String format, Schema schema) + throws Exception { + String topicName = newTopicName(); + TestingUser expectedUser = createRandomUser(); + pulsar.operator().sendMessage(topicName, schema, expectedUser); + + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + + final List result = + collectRows( + tableEnv.sqlQuery( + String.format( + "select * from %s ", + TopicName.get(topicName).getLocalName())), + 1); + assertThat(result) + .containsExactlyElementsOf( + Collections.singletonList( + Row.of(expectedUser.getAge(), expectedUser.getName()))); + } + + @ParameterizedTest + @MethodSource("provideAvroBasedSchemaData") + void readFromExplicitTableWithAvroBasedSchema(String format, Schema schema) + throws Exception { + // TODO add this test + String databaseName = newDatabaseName(); + String topicName = newTopicName(); + TestingUser expectedUser = createRandomUser(); + pulsar.operator().sendMessage(topicName, schema, expectedUser); + + tableEnv.useCatalog(PULSAR_CATALOG1); + + String dbDDL = "CREATE DATABASE " + databaseName; + tableEnv.executeSql(dbDDL).print(); + tableEnv.useDatabase(databaseName); + + String sourceTableDDL = + String.format( + "CREATE TABLE %s (\n" + + " age INT,\n" + + " name STRING\n" + + ") with (\n" + + " 'connector' = 'pulsar',\n" + + " 'topics' = '%s',\n" + + " 'format' = '%s'\n" + + ")", + topicName, topicName, format); + tableEnv.executeSql(sourceTableDDL).await(); + + final List result = + collectRows( + tableEnv.sqlQuery( + String.format( + "select * from %s ", + TopicName.get(topicName).getLocalName())), + 1); + assertThat(result) + .containsExactlyElementsOf( + Collections.singletonList( + Row.of(expectedUser.getAge(), expectedUser.getName()))); + } + + @Test + void readFromNativeTableWithStringSchemaUsingRawFormat() throws Exception { + String topicName = newTopicName(); + String expectedString = "expected_string"; + pulsar.operator().sendMessage(topicName, Schema.STRING, expectedString); + + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + + final List result = + collectRows( + tableEnv.sqlQuery( + String.format( + "select * from %s", + TopicName.get(topicName).getLocalName())), + 1); + assertThat(result) + .containsExactlyElementsOf(Collections.singletonList(Row.of(expectedString))); + } + + @Test + void readFromNativeTableWithComplexSchemaUsingRawFormatShouldFail() { + String topicName = newTopicName(); + TestingUser expectedUser = createRandomUser(); + pulsar.operator().sendMessage(topicName, Schema.AVRO(TestingUser.class), expectedUser); + + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + + assertThatExceptionOfType(ValidationException.class) + .isThrownBy( + () -> + collectRows( + tableEnv.sqlQuery( + String.format( + "select * from %s /*+ OPTIONS('format'='raw') */", + TopicName.get(topicName).getLocalName())), + 1)) + .withMessageStartingWith("The 'raw' format only supports single physical column."); + } + + @Test + void copyDataFromNativeTableToNativeTable() throws Exception { + String sourceTopic = newTopicName(); + String sourceTableName = TopicName.get(sourceTopic).getLocalName(); + pulsar.operator().sendMessages(sourceTopic, Schema.INT32, INTEGER_LIST); + + String sinkTopic = newTopicName(); + String sinkTableName = TopicName.get(sinkTopic).getLocalName(); + pulsar.operator().createTopic(sinkTopic, 1); + pulsar.operator().admin().schemas().createSchema(sinkTopic, Schema.INT32.getSchemaInfo()); + + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + String insertQ = + String.format("INSERT INTO %s SELECT * FROM %s", sinkTableName, sourceTableName); + + tableEnv.executeSql(insertQ); + List result = + pulsar.operator().receiveMessages(sinkTopic, Schema.INT32, INTEGER_LIST.size()) + .stream() + .map(Message::getValue) + .collect(Collectors.toList()); + assertThat(result).containsExactlyElementsOf(INTEGER_LIST); + } + + @ParameterizedTest + @MethodSource("provideAvroBasedSchemaData") + void writeToExplicitTableAndReadWithAvroBasedSchema(String format, Schema schema) + throws Exception { + String databaseName = newDatabaseName(); + String tableSinkTopic = newTopicName(); + String tableSinkName = TopicName.get(tableSinkTopic).getLocalName(); + + pulsar.operator().createTopic(tableSinkTopic, 1); + tableEnv.useCatalog(PULSAR_CATALOG1); + + String dbDDL = "CREATE DATABASE " + databaseName; + tableEnv.executeSql(dbDDL).print(); + tableEnv.useDatabase(databaseName); + + String sinkDDL = + String.format( + "CREATE TABLE %s (\n" + + " oid STRING,\n" + + " totalprice INT,\n" + + " customerid STRING\n" + + ") with (\n" + + " 'connector' = 'pulsar',\n" + + " 'topics' = '%s',\n" + + " 'format' = '%s'\n" + + ")", + tableSinkName, tableSinkTopic, format); + tableEnv.executeSql(sinkDDL).await(); + + String insertQ = + String.format( + "INSERT INTO %s" + + " VALUES\n" + + " ('oid1', 10, 'cid1'),\n" + + " ('oid2', 20, 'cid2'),\n" + + " ('oid3', 30, 'cid3'),\n" + + " ('oid4', 10, 'cid4')", + tableSinkName); + tableEnv.executeSql(insertQ).await(); + + final List result = + collectRows(tableEnv.sqlQuery("select * from " + tableSinkName), 4); + assertThat(result).hasSize(4); + } + + @ParameterizedTest + @MethodSource("provideAvroBasedSchemaData") + void writeToExplicitTableAndReadWithAvroBasedSchemaUsingPulsarConsumer( + String format, Schema schema) throws Exception { + String databaseName = newDatabaseName(); + String tableSinkTopic = newTopicName(); + String tableSinkName = TopicName.get(tableSinkTopic).getLocalName(); + + pulsar.operator().createTopic(tableSinkTopic, 1); + tableEnv.useCatalog(PULSAR_CATALOG1); + + String dbDDL = "CREATE DATABASE " + databaseName; + tableEnv.executeSql(dbDDL).print(); + tableEnv.executeSql("USE " + databaseName + ""); + + String sinkDDL = + String.format( + "CREATE TABLE %s (\n" + + " name STRING,\n" + + " age INT\n" + + ") with (\n" + + " 'connector' = 'pulsar',\n" + + " 'topics' = '%s',\n" + + " 'format' = '%s'\n" + + ")", + tableSinkName, tableSinkTopic, format); + tableEnv.executeSql(sinkDDL).await(); + + String insertQ = + String.format( + "INSERT INTO %s" + + " VALUES\n" + + " ('oid1', 10),\n" + + " ('oid2', 20),\n" + + " ('oid3', 30),\n" + + " ('oid4', 40)", + tableSinkName); + tableEnv.executeSql(insertQ).await(); + + List sinkResult = + pulsar.operator().receiveMessages(tableSinkTopic, schema, 4).stream() + .map(Message::getValue) + .collect(Collectors.toList()); + assertThat(sinkResult) + .containsExactly( + createUser("oid1", 10), + createUser("oid2", 20), + createUser("oid3", 30), + createUser("oid4", 40)); + } + + @ParameterizedTest + @MethodSource("provideAvroBasedSchemaData") + void writeToNativeTableAndReadWithAvroBasedSchema(String format, Schema schema) + throws Exception { + String tableSinkTopic = newTopicName(); + + pulsar.operator().createTopic(tableSinkTopic, 1); + pulsar.operator().admin().schemas().createSchema(tableSinkTopic, schema.getSchemaInfo()); + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + + String insertQ = + String.format( + "INSERT INTO %s" + + " VALUES\n" + + " (1, 'abc'),\n" + + " (2, 'bcd'),\n" + + " (3, 'cde'),\n" + + " (4, 'def')", + tableSinkTopic); + tableEnv.executeSql(insertQ).await(); + + final List result = + collectRows( + tableEnv.sqlQuery(String.format("SELECT * FROM %s", tableSinkTopic)), 4); + assertThat(result).hasSize(4); + } + + @Test + void writeToNativeTableAndReadWithStringSchema() throws Exception { + String tableSinkTopic = newTopicName(); + + pulsar.operator().createTopic(tableSinkTopic, 1); + pulsar.operator() + .admin() + .schemas() + .createSchema(tableSinkTopic, Schema.STRING.getSchemaInfo()); + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + + String insertQ = + String.format( + "INSERT INTO %s" + + " VALUES\n" + + " ('abc'),\n" + + " ('bcd'),\n" + + " ('cde'),\n" + + " ('def')", + tableSinkTopic); + tableEnv.executeSql(insertQ).await(); + + final List result = + collectRows( + tableEnv.sqlQuery(String.format("SELECT * FROM %s", tableSinkTopic)), 4); + assertThat(result).hasSize(4); + } + + @Test + void writeToNativeTableAndReadWithIntegerSchema() throws Exception { + String tableSinkTopic = newTopicName(); + + pulsar.operator().createTopic(tableSinkTopic, 1); + pulsar.operator() + .admin() + .schemas() + .createSchema(tableSinkTopic, Schema.INT32.getSchemaInfo()); + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + + String insertQ = + String.format( + "INSERT INTO %s" + + " VALUES\n" + + " (1),\n" + + " (2),\n" + + " (3),\n" + + " (4)", + tableSinkTopic); + tableEnv.executeSql(insertQ).await(); + + final List result = + collectRows( + tableEnv.sqlQuery(String.format("SELECT * FROM %s", tableSinkTopic)), 4); + assertThat(result).hasSize(4); + } + + @Test + void writeToNativeTableAndReadWithIntegerSchemaUsingValueField() throws Exception { + String tableSinkTopic = newTopicName(); + + pulsar.operator().createTopic(tableSinkTopic, 1); + pulsar.operator() + .admin() + .schemas() + .createSchema(tableSinkTopic, Schema.INT32.getSchemaInfo()); + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + + String insertQ = + String.format( + "INSERT INTO %s" + + " VALUES\n" + + " (1),\n" + + " (2),\n" + + " (3),\n" + + " (4)", + tableSinkTopic); + tableEnv.executeSql(insertQ).await(); + + final List result = + collectRows( + tableEnv.sqlQuery(String.format("SELECT `value` FROM %s", tableSinkTopic)), + 4); + assertThat(result).hasSize(4); + } + + // utils + private void registerCatalogs(TableEnvironment tableEnvironment) { + tableEnvironment.registerCatalog( + INMEMORY_CATALOG, new GenericInMemoryCatalog(INMEMORY_CATALOG, INMEMORY_DB)); + + PulsarConfigBuilder configBuilder = new PulsarConfigBuilder(); + configBuilder.set(PulsarOptions.PULSAR_ADMIN_URL, pulsar.operator().adminUrl()); + configBuilder.set(PulsarOptions.PULSAR_SERVICE_URL, pulsar.operator().serviceUrl()); + tableEnvironment.registerCatalog( + PULSAR_CATALOG1, + new PulsarCatalog( + PULSAR_CATALOG1, + configBuilder.build( + CATALOG_CONFIG_VALIDATOR, PulsarCatalogConfiguration::new), + PULSAR1_DB, + FLINK_TENANT)); + + tableEnvironment.registerCatalog( + PULSAR_CATALOG2, + new PulsarCatalog( + PULSAR_CATALOG2, + configBuilder.build( + CATALOG_CONFIG_VALIDATOR, PulsarCatalogConfiguration::new), + PULSAR2_DB, + FLINK_TENANT)); + + tableEnvironment.useCatalog(INMEMORY_CATALOG); + } + + private String newDatabaseName() { + return "database" + RandomStringUtils.randomNumeric(8); + } + + private String newTopicName() { + return RandomStringUtils.randomAlphabetic(5); + } + + private static Stream provideAvroBasedSchemaData() { + return Stream.of( + Arguments.of(JSON_FORMAT, Schema.JSON(TestingUser.class)), + Arguments.of(AVRO_FORMAT, Schema.AVRO(TestingUser.class))); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockPulsarAuthentication.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockPulsarAuthentication.java new file mode 100644 index 0000000000000..cbb2a867f19b3 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockPulsarAuthentication.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.testutils; + +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationDataProvider; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.impl.auth.AuthenticationDataNull; + +import java.io.IOException; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.CompletableFuture; + +/** A class to verify Pulsar authentication auth params map is created properly. way to do this */ +public class MockPulsarAuthentication implements Authentication { + public static String key1 = "key1"; + public static String key2 = "key2"; + public static String value1 = "value1"; + public static String value2 = "value2"; + + @Override + public String getAuthMethodName() { + return "custom authentication"; + } + + @Override + public AuthenticationDataProvider getAuthData() { + return new AuthenticationDataNull(); + } + + @Override + public AuthenticationDataProvider getAuthData(String brokerHostName) { + return new AuthenticationDataNull(); + } + + @Override + public void authenticationStage( + String requestUrl, + AuthenticationDataProvider authData, + Map previousResHeaders, + CompletableFuture> authFuture) { + Authentication.super.authenticationStage( + requestUrl, authData, previousResHeaders, authFuture); + } + + @Override + public Set> newRequestHeader( + String hostName, + AuthenticationDataProvider authData, + Map previousResHeaders) { + return new HashSet<>(); + } + + @Override + public void configure(Map authParams) { + assert Objects.equals(authParams.get(key1), value1); + assert Objects.equals(authParams.get(key2), value2); + } + + @Override + public void start() throws PulsarClientException {} + + @Override + public void close() throws IOException {} +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/TimestampStopCursor.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockTopicRouter.java similarity index 57% rename from flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/TimestampStopCursor.java rename to flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockTopicRouter.java index 534e77f54b78e..b5ef53841aaee 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/TimestampStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockTopicRouter.java @@ -16,24 +16,22 @@ * limitations under the License. */ -package org.apache.flink.connector.pulsar.source.enumerator.cursor.stop; +package org.apache.flink.connector.pulsar.table.testutils; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.sink.writer.context.PulsarSinkContext; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; +import org.apache.flink.table.data.RowData; -import org.apache.pulsar.client.api.Message; +import java.util.List; -/** Stop consuming message at the given event time. */ -public class TimestampStopCursor implements StopCursor { - private static final long serialVersionUID = 3381576769339353027L; +/** A mock topic Router for testing purposes only. */ +public class MockTopicRouter implements TopicRouter { - private final long timestamp; - - public TimestampStopCursor(long timestamp) { - this.timestamp = timestamp; - } + private static final long serialVersionUID = 1316133122715449818L; @Override - public boolean shouldStop(Message message) { - return message.getEventTime() >= timestamp; + public String route( + RowData rowData, String key, List partitions, PulsarSinkContext context) { + return "never-exist-topic"; } } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/PulsarTableTestUtils.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/PulsarTableTestUtils.java new file mode 100644 index 0000000000000..f43be2d63eebf --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/PulsarTableTestUtils.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.testutils; + +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; + +/** Util class for verify testing results. */ +public class PulsarTableTestUtils { + public static List collectRows(Table table, int expectedSize) throws Exception { + final TableResult result = table.execute(); + final List collectedRows = new ArrayList<>(); + try (CloseableIterator iterator = result.collect()) { + while (collectedRows.size() < expectedSize && iterator.hasNext()) { + collectedRows.add(iterator.next()); + } + } + result.getJobClient() + .ifPresent( + jc -> { + try { + jc.cancel().get(5, TimeUnit.SECONDS); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + + return collectedRows; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/SchemaData.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/SchemaData.java new file mode 100644 index 0000000000000..9143af84a3eca --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/SchemaData.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.testutils; + +import java.util.Arrays; +import java.util.List; + +/** Data for various test cases. New test data can be added here */ +public class SchemaData { + public static final List INTEGER_LIST = Arrays.asList(1, 2, 3, 4, 5); +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/TestingUser.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/TestingUser.java new file mode 100644 index 0000000000000..b597f3fa3d875 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/TestingUser.java @@ -0,0 +1,61 @@ +package org.apache.flink.connector.pulsar.table.testutils; + +import java.io.Serializable; +import java.util.Objects; +import java.util.concurrent.ThreadLocalRandom; + +import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; + +/** A test POJO class. */ +public class TestingUser implements Serializable { + private static final long serialVersionUID = -1123545861004770003L; + public String name; + public Integer age; + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public Integer getAge() { + return age; + } + + public void setAge(Integer age) { + this.age = age; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TestingUser that = (TestingUser) o; + return Objects.equals(name, that.name) && Objects.equals(age, that.age); + } + + @Override + public int hashCode() { + return Objects.hash(name, age); + } + + public static TestingUser createUser(String name, Integer age) { + TestingUser user = new TestingUser(); + user.setAge(age); + user.setName(name); + return user; + } + + public static TestingUser createRandomUser() { + TestingUser user = new TestingUser(); + user.setName(randomAlphabetic(5)); + user.setAge(ThreadLocalRandom.current().nextInt(0, Integer.MAX_VALUE)); + return user; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestCommonUtils.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestCommonUtils.java index 87f3976c6a74c..96b1ca62f2a06 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestCommonUtils.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestCommonUtils.java @@ -19,10 +19,12 @@ package org.apache.flink.connector.pulsar.testutils; import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; +import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.pulsar.client.api.MessageId; import org.junit.jupiter.api.extension.ParameterContext; @@ -33,7 +35,18 @@ /** Put static methods that can be used by multiple test classes. */ public class PulsarTestCommonUtils { - // ------- CreateSplits + /** Convert the CheckpointingMode to a connector related DeliveryGuarantee. */ + public static DeliveryGuarantee toDeliveryGuarantee(CheckpointingMode checkpointingMode) { + if (checkpointingMode == CheckpointingMode.AT_LEAST_ONCE) { + return DeliveryGuarantee.AT_LEAST_ONCE; + } else if (checkpointingMode == CheckpointingMode.EXACTLY_ONCE) { + return DeliveryGuarantee.EXACTLY_ONCE; + } else { + throw new IllegalArgumentException( + "Only exactly-once and al-least-once checkpointing mode are supported."); + } + } + /** creates a fullRange() partitionSplit. */ public static PulsarPartitionSplit createPartitionSplit(String topic, int partitionId) { return createPartitionSplit(topic, partitionId, Boundedness.CONTINUOUS_UNBOUNDED); @@ -63,8 +76,6 @@ public static List createPartitionSplits( return splits; } - // -------- InvocationContext Utils - public static boolean isAssignableFromParameterContext( Class requiredType, ParameterContext context) { return requiredType.isAssignableFrom(context.getParameter().getType()); diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContext.java index f238a03bfa587..dd7a3fbfbe420 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContext.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContext.java @@ -19,37 +19,46 @@ package org.apache.flink.connector.pulsar.testutils; import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator; -import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext; +import org.apache.flink.connector.testframe.external.ExternalContext; + +import org.apache.pulsar.client.api.Schema; import java.net.URL; -import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; import java.util.List; +import java.util.Set; + +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicName; + +/** + * The implementation for Flink connector test tools. Providing the common test case writing + * constraint for both source, sink and table API. + */ +public abstract class PulsarTestContext implements ExternalContext { -/** Common test context for pulsar based test. */ -public abstract class PulsarTestContext implements DataStreamSourceExternalContext { + private final Set generateTopics = new HashSet<>(); protected final PulsarRuntimeOperator operator; - protected final List connectorJarPaths; + // The schema used for consuming and producing messages between Pulsar and tests. + protected final Schema schema; - protected PulsarTestContext(PulsarTestEnvironment environment, List connectorJarPaths) { + protected PulsarTestContext(PulsarTestEnvironment environment, Schema schema) { this.operator = environment.operator(); - this.connectorJarPaths = connectorJarPaths; + this.schema = schema; } - // Helper methods for generating data. - - protected List generateStringTestData(int splitIndex, long seed) { - int recordNum = 300; - List records = new ArrayList<>(recordNum); - for (int i = 0; i < recordNum; i++) { - records.add(splitIndex + "-" + i); - } + /** Implement this method for providing a more friendly test name in IDE. */ + protected abstract String displayName(); - return records; + /** + * Add the generated topic into the testing context, They would be cleaned after all the cases + * have finished. + */ + protected final void registerTopic(String topic) { + generateTopics.add(topicName(topic)); } - protected abstract String displayName(); - @Override public String toString() { return displayName(); @@ -57,6 +66,14 @@ public String toString() { @Override public List getConnectorJarPaths() { - return connectorJarPaths; + // We don't need any tests jar definition. They are provided in docker related environments. + return Collections.emptyList(); + } + + @Override + public void close() throws Exception { + for (String topic : generateTopics) { + operator.deleteTopic(topic); + } } } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContextFactory.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContextFactory.java index c634efffc44c1..3c888f59d196d 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContextFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContextFactory.java @@ -18,13 +18,16 @@ package org.apache.flink.connector.pulsar.testutils; +import org.apache.flink.connector.pulsar.testutils.sink.PulsarSinkTestContext; +import org.apache.flink.connector.pulsar.testutils.source.PulsarSourceTestContext; import org.apache.flink.connector.testframe.external.ExternalContextFactory; import java.util.function.Function; /** - * Factory for creating all the test context that extends {@link PulsarTestContext}. Test context - * class should have a constructor with {@link PulsarTestEnvironment} arg. + * Factory for creating all the test context that extends {@link PulsarSourceTestContext} or {@link + * PulsarSinkTestContext}. Test context class should have a constructor with single {@link + * PulsarTestEnvironment} arg. */ public class PulsarTestContextFactory> implements ExternalContextFactory { diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestEnvironment.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestEnvironment.java index 0f3fb9e15f9c5..f921e4b92f158 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestEnvironment.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestEnvironment.java @@ -48,7 +48,7 @@ * }

* *

If you want to use this class in JUnit 5, just simply extends {@link PulsarTestSuiteBase}, all - * the helper methods in {@code PulsarContainerOperator} is also exposed there. + * the helper methods in {@link PulsarRuntimeOperator} is also exposed there. */ public class PulsarTestEnvironment implements BeforeAllCallback, AfterAllCallback, TestResource, TestRule { diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestSuiteBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestSuiteBase.java index c87140ba42762..c8ef72ca4c682 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestSuiteBase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestSuiteBase.java @@ -30,9 +30,9 @@ * The base class for the all Pulsar related test sites. It brings up: * *

    - *
  • A Zookeeper cluster. - *
  • Pulsar Broker. - *
  • A Bookkeeper cluster. + *
  • A standalone Zookeeper. + *
  • A standalone Pulsar Broker. + *
  • A standalone Bookkeeper. *
* *

You just need to write a JUnit 5 test class and extends this suite class. All the helper @@ -56,7 +56,7 @@ public abstract class PulsarTestSuiteBase { * pulsar broker. Override this method when needs. */ protected PulsarRuntime runtime() { - return PulsarRuntime.embedded(); + return PulsarRuntime.container(); } /** Operate pulsar by acquiring a runtime operator. */ diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/MultipleTopicTemplateContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/MultipleTopicTemplateContext.java deleted file mode 100644 index 3eca9e7f92636..0000000000000 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/MultipleTopicTemplateContext.java +++ /dev/null @@ -1,134 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.connector.pulsar.testutils.cases; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.api.connector.source.Source; -import org.apache.flink.connector.pulsar.source.PulsarSource; -import org.apache.flink.connector.pulsar.source.PulsarSourceBuilder; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; -import org.apache.flink.connector.pulsar.testutils.PulsarPartitionDataWriter; -import org.apache.flink.connector.pulsar.testutils.PulsarTestContext; -import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; -import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter; -import org.apache.flink.connector.testframe.external.source.TestingSourceSettings; - -import org.apache.pulsar.client.api.RegexSubscriptionMode; -import org.apache.pulsar.client.api.SubscriptionType; - -import java.net.URL; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; -import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.pulsarSchema; -import static org.apache.pulsar.client.api.Schema.STRING; - -/** - * Pulsar external context template that will create multiple topics with only one partitions as - * source splits. - */ -public abstract class MultipleTopicTemplateContext extends PulsarTestContext { - - private int numTopics = 0; - - private final String topicPattern = "pulsar-multiple-topic-[0-9]+-" + randomAlphabetic(8); - - private final Map> topicNameToSplitWriters = - new HashMap<>(); - - public MultipleTopicTemplateContext(PulsarTestEnvironment environment) { - this(environment, Collections.emptyList()); - } - - public MultipleTopicTemplateContext( - PulsarTestEnvironment environment, List connectorJarPaths) { - super(environment, connectorJarPaths); - } - - @Override - public Source createSource(TestingSourceSettings sourceSettings) { - PulsarSourceBuilder builder = - PulsarSource.builder() - .setDeserializationSchema(pulsarSchema(STRING)) - .setServiceUrl(serviceUrl()) - .setAdminUrl(adminUrl()) - .setTopicPattern(topicPattern, RegexSubscriptionMode.AllTopics) - .setSubscriptionType(subscriptionType()) - .setSubscriptionName(subscriptionName()); - if (sourceSettings.getBoundedness() == Boundedness.BOUNDED) { - // Using latest stop cursor for making sure the source could be stopped. - // This is required for SourceTestSuiteBase. - builder.setBoundedStopCursor(StopCursor.latest()); - } - - return builder.build(); - } - - @Override - public ExternalSystemSplitDataWriter createSourceSplitDataWriter( - TestingSourceSettings sourceSettings) { - String topicName = topicPattern.replace("[0-9]+", String.valueOf(numTopics)); - operator.createTopic(topicName, 1); - - String partitionName = TopicNameUtils.topicNameWithPartition(topicName, 0); - PulsarPartitionDataWriter writer = new PulsarPartitionDataWriter(operator, partitionName); - - topicNameToSplitWriters.put(partitionName, writer); - numTopics++; - - return writer; - } - - @Override - public List generateTestData( - TestingSourceSettings sourceSettings, int splitIndex, long seed) { - return generateStringTestData(splitIndex, seed); - } - - @Override - public TypeInformation getProducedType() { - return TypeInformation.of(String.class); - } - - @Override - public void close() throws Exception { - for (ExternalSystemSplitDataWriter writer : topicNameToSplitWriters.values()) { - writer.close(); - } - - topicNameToSplitWriters.clear(); - } - - protected abstract String subscriptionName(); - - protected abstract SubscriptionType subscriptionType(); - - protected String serviceUrl() { - return operator.serviceUrl(); - } - - protected String adminUrl() { - return operator.adminUrl(); - } -} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/SingleTopicConsumingContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/SingleTopicConsumingContext.java deleted file mode 100644 index f5bfa45f32b7a..0000000000000 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/SingleTopicConsumingContext.java +++ /dev/null @@ -1,132 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.connector.pulsar.testutils.cases; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.api.connector.source.Source; -import org.apache.flink.connector.pulsar.source.PulsarSource; -import org.apache.flink.connector.pulsar.source.PulsarSourceBuilder; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; -import org.apache.flink.connector.pulsar.testutils.PulsarPartitionDataWriter; -import org.apache.flink.connector.pulsar.testutils.PulsarTestContext; -import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; -import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter; -import org.apache.flink.connector.testframe.external.source.TestingSourceSettings; - -import java.net.URL; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ThreadLocalRandom; - -import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.pulsarSchema; -import static org.apache.pulsar.client.api.Schema.STRING; -import static org.apache.pulsar.client.api.SubscriptionType.Exclusive; - -/** - * A Pulsar external context that will create only one topic and use partitions in that topic as - * source splits. - */ -public class SingleTopicConsumingContext extends PulsarTestContext { - - private static final String TOPIC_NAME_PREFIX = "pulsar-single-topic"; - private final String topicName; - private final Map> partitionToSplitWriter = - new HashMap<>(); - - private int numSplits = 0; - - public SingleTopicConsumingContext(PulsarTestEnvironment environment) { - this(environment, Collections.emptyList()); - } - - public SingleTopicConsumingContext( - PulsarTestEnvironment environment, List connectorJarPaths) { - super(environment, connectorJarPaths); - this.topicName = - TOPIC_NAME_PREFIX + "-" + ThreadLocalRandom.current().nextLong(Long.MAX_VALUE); - } - - @Override - protected String displayName() { - return "consuming message on single topic"; - } - - @Override - public Source createSource(TestingSourceSettings sourceSettings) { - PulsarSourceBuilder builder = - PulsarSource.builder() - .setDeserializationSchema(pulsarSchema(STRING)) - .setServiceUrl(operator.serviceUrl()) - .setAdminUrl(operator.adminUrl()) - .setTopics(topicName) - .setSubscriptionType(Exclusive) - .setSubscriptionName("pulsar-single-topic"); - if (sourceSettings.getBoundedness() == Boundedness.BOUNDED) { - // Using latest stop cursor for making sure the source could be stopped. - // This is required for SourceTestSuiteBase. - builder.setBoundedStopCursor(StopCursor.latest()); - } - - return builder.build(); - } - - @Override - public ExternalSystemSplitDataWriter createSourceSplitDataWriter( - TestingSourceSettings sourceSettings) { - if (numSplits == 0) { - // Create the topic first. - operator.createTopic(topicName, 1); - numSplits++; - } else { - numSplits++; - operator.increaseTopicPartitions(topicName, numSplits); - } - - String partitionName = TopicNameUtils.topicNameWithPartition(topicName, numSplits - 1); - PulsarPartitionDataWriter writer = new PulsarPartitionDataWriter(operator, partitionName); - partitionToSplitWriter.put(numSplits - 1, writer); - - return writer; - } - - @Override - public List generateTestData( - TestingSourceSettings sourceSettings, int splitIndex, long seed) { - return generateStringTestData(splitIndex, seed); - } - - @Override - public TypeInformation getProducedType() { - return TypeInformation.of(String.class); - } - - @Override - public void close() throws Exception { - // Close writer. - for (ExternalSystemSplitDataWriter writer : partitionToSplitWriter.values()) { - writer.close(); - } - - partitionToSplitWriter.clear(); - } -} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntime.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntime.java index 9c1cd01a7f310..613ad270e8733 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntime.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntime.java @@ -20,7 +20,7 @@ import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; import org.apache.flink.connector.pulsar.testutils.runtime.container.PulsarContainerRuntime; -import org.apache.flink.connector.pulsar.testutils.runtime.embedded.PulsarEmbeddedRuntime; +import org.apache.flink.connector.pulsar.testutils.runtime.mock.PulsarMockRuntime; import org.testcontainers.containers.GenericContainer; @@ -44,38 +44,41 @@ public interface PulsarRuntime { */ PulsarRuntimeOperator operator(); + /** Create a Pulsar instance which would mock all the backends. */ + static PulsarRuntime mock() { + return new PulsarMockRuntime(); + } + /** - * Create a standalone Pulsar instance in test thread. We would start a embedded zookeeper and - * bookkeeper. The stream storage for bookkeeper is disabled. The function worker is disabled on - * Pulsar broker. - * - *

This runtime would be faster than {@link #container()} and behaves the same like the - * {@link #container()}. + * Create a Pulsar instance in docker. We would start a standalone Pulsar in TestContainers. + * This runtime is often used in end-to-end tests. The performance may be a bit of slower than + * {@link #mock()}. The stream storage for bookkeeper is disabled. The function worker is + * disabled on Pulsar broker. */ - static PulsarRuntime embedded() { - return new PulsarEmbeddedRuntime(); + static PulsarRuntime container() { + return new PulsarContainerRuntime(false); } /** * Create a Pulsar instance in docker. We would start a standalone Pulsar in TestContainers. * This runtime is often used in end-to-end tests. The performance may be a bit of slower than - * {@link #embedded()}. The stream storage for bookkeeper is disabled. The function worker is + * {@link #mock()}. The stream storage for bookkeeper is disabled. The function worker is * disabled on Pulsar broker. */ - static PulsarRuntime container() { - return new PulsarContainerRuntime(); + static PulsarRuntime authContainer() { + return new PulsarContainerRuntime(true); } /** * Create a Pulsar instance in docker. We would start a standalone Pulsar in TestContainers. * This runtime is often used in end-to-end tests. The performance may be a bit of slower than - * {@link #embedded()}. The stream storage for bookkeeper is disabled. The function worker is + * {@link #mock()}. The stream storage for bookkeeper is disabled. The function worker is * disabled on Pulsar broker. * *

We would link the created Pulsar docker instance with the given flink instance. This would * enable the connection for Pulsar and Flink in docker environment. */ static PulsarRuntime container(GenericContainer flinkContainer) { - return new PulsarContainerRuntime().bindWithFlinkContainer(flinkContainer); + return new PulsarContainerRuntime(false).bindWithFlinkContainer(flinkContainer); } } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeOperator.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeOperator.java index a78ea992d4b8a..618e879eeb577 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeOperator.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeOperator.java @@ -21,6 +21,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.connector.pulsar.common.config.PulsarConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange; import org.apache.flink.connector.testframe.external.ExternalContext; @@ -31,13 +32,14 @@ import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.admin.PulsarAdminException.NotFoundException; import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.ConsumerBuilder; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerBuilder; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.api.TypedMessageBuilder; import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient; import org.apache.pulsar.client.api.transaction.TxnID; @@ -49,10 +51,8 @@ import java.io.IOException; import java.time.Duration; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.List; -import java.util.Map; import java.util.Random; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; @@ -62,9 +62,7 @@ import static java.util.Collections.emptyList; import static java.util.Collections.singletonList; import static java.util.concurrent.TimeUnit.MILLISECONDS; -import static java.util.function.Function.identity; import static java.util.stream.Collectors.toList; -import static java.util.stream.Collectors.toMap; import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; import static org.apache.flink.connector.base.DeliveryGuarantee.EXACTLY_ONCE; import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL; @@ -79,8 +77,11 @@ import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicName; import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.pulsar.client.api.SubscriptionInitialPosition.Earliest; import static org.apache.pulsar.client.api.SubscriptionMode.Durable; import static org.apache.pulsar.client.api.SubscriptionType.Exclusive; +import static org.apache.pulsar.common.naming.TopicDomain.persistent; +import static org.apache.pulsar.common.partition.PartitionedTopicMetadata.NON_PARTITIONED; /** * A pulsar cluster operator used for operating pulsar instance. It's serializable for using in @@ -122,6 +123,16 @@ public PulsarRuntimeOperator( this.consumers = new ConcurrentHashMap<>(); } + public boolean topicExists(String topic) { + TopicName topicName = TopicName.get(topic); + String namespace = topicName.getNamespace(); + String parsedTopic = topicName.toString(); + + return sneakyAdmin(() -> admin().topics().getList(namespace, persistent)).stream() + .map(TopicNameUtils::topicName) + .anyMatch(name -> name.equals(parsedTopic)); + } + /** * Create a topic with default {@link #DEFAULT_PARTITIONS} partitions and send a fixed number * {@link #NUM_RECORDS_PER_PARTITION} of records to this topic. @@ -178,10 +189,12 @@ public void setupTopic( */ public void createTopic(String topic, int numberOfPartitions) { checkArgument(numberOfPartitions >= 0); - if (numberOfPartitions <= 0) { - createNonPartitionedTopic(topic); + checkArgument(!topicExists(topic), "Topic %s exists.", topic); + + if (numberOfPartitions == 0) { + sneakyAdmin(() -> admin().topics().createNonPartitionedTopic(topic)); } else { - createPartitionedTopic(topic, numberOfPartitions); + sneakyAdmin(() -> admin().topics().createPartitionedTopic(topic, numberOfPartitions)); } } @@ -196,7 +209,7 @@ public void increaseTopicPartitions(String topic, int newPartitionsNum) { sneakyAdmin(() -> admin().topics().getPartitionedTopicMetadata(topic)); checkArgument( metadata.partitions < newPartitionsNum, - "The new partition size which should exceed previous size."); + "The new partition size which should greater than previous size."); sneakyAdmin(() -> admin().topics().updatePartitionedTopic(topic, newPartitionsNum)); } @@ -220,9 +233,11 @@ public void deleteTopic(String topic) { return; } + // Close all the available consumers and producers. removeConsumers(topic); removeProducers(topic); - if (metadata.partitions <= 0) { + + if (metadata.partitions == NON_PARTITIONED) { sneakyAdmin(() -> admin().topics().delete(topicName)); } else { sneakyAdmin(() -> admin().topics().deletePartitionedTopic(topicName)); @@ -245,22 +260,6 @@ public List topicInfo(String topic) { } } - /** - * Query a list of topics. Convert the topic metadata into a list of topic partitions. Return a - * mapping for topic and its partitions. - */ - public Map> topicsInfo(String... topics) { - return topicsInfo(Arrays.asList(topics)); - } - - /** - * Query a list of topics. Convert the topic metadata into a list of topic partitions. Return a - * mapping for topic and its partitions. - */ - public Map> topicsInfo(Collection topics) { - return topics.stream().collect(toMap(identity(), this::topicInfo)); - } - /** * Send a single message to Pulsar, return the message id after the ack from Pulsar. * @@ -361,8 +360,12 @@ public Message receiveMessage(String topic, Schema schema) { public Message receiveMessage(String topic, Schema schema, Duration timeout) { try { Consumer consumer = createConsumer(topic, schema); - Message message = consumer.receiveAsync().get(timeout.toMillis(), MILLISECONDS); - consumer.acknowledgeCumulative(message.getMessageId()); + int millis = Math.toIntExact(timeout.toMillis()); + Message message = consumer.receive(millis, MILLISECONDS); + + if (message != null) { + consumer.acknowledgeCumulative(message.getMessageId()); + } return message; } catch (Exception e) { @@ -487,26 +490,8 @@ public void close() throws IOException { // --------------------------- Private Methods ----------------------------- - private void createNonPartitionedTopic(String topic) { - try { - admin().lookups().lookupTopic(topic); - sneakyAdmin(() -> admin().topics().expireMessagesForAllSubscriptions(topic, 0)); - } catch (PulsarAdminException e) { - sneakyAdmin(() -> admin().topics().createNonPartitionedTopic(topic)); - } - } - - private void createPartitionedTopic(String topic, int numberOfPartitions) { - try { - admin().lookups().lookupPartitionedTopic(topic); - sneakyAdmin(() -> admin().topics().expireMessagesForAllSubscriptions(topic, 0)); - } catch (PulsarAdminException e) { - sneakyAdmin(() -> admin().topics().createPartitionedTopic(topic, numberOfPartitions)); - } - } - @SuppressWarnings("unchecked") - private Producer createProducer(String topic, Schema schema) + public Producer createProducer(String topic, Schema schema) throws PulsarClientException { TopicName topicName = TopicName.get(topic); String name = topicName.getPartitionedTopicName(); @@ -518,12 +503,13 @@ private Producer createProducer(String topic, Schema schema) topicProducers.computeIfAbsent( index, i -> { - try { - return client().newProducer(schema).topic(topic).create(); - } catch (PulsarClientException e) { - sneakyThrow(e); - return null; - } + ProducerBuilder builder = + client().newProducer(schema) + .topic(topic) + .enableBatching(false) + .enableMultiSchema(true); + + return sneakyClient(builder::create); }); } @@ -540,19 +526,15 @@ private Consumer createConsumer(String topic, Schema schema) topicConsumers.computeIfAbsent( index, i -> { - try { - return client().newConsumer(schema) - .topic(topic) - .subscriptionName(SUBSCRIPTION_NAME) - .subscriptionMode(Durable) - .subscriptionType(Exclusive) - .subscriptionInitialPosition( - SubscriptionInitialPosition.Earliest) - .subscribe(); - } catch (PulsarClientException e) { - sneakyThrow(e); - return null; - } + ConsumerBuilder builder = + client().newConsumer(schema) + .topic(topic) + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionMode(Durable) + .subscriptionType(Exclusive) + .subscriptionInitialPosition(Earliest); + + return sneakyClient(builder::subscribe); }); } @@ -561,11 +543,7 @@ private void removeProducers(String topic) { ConcurrentHashMap> integerProducers = producers.remove(topicName); if (integerProducers != null) { for (Producer producer : integerProducers.values()) { - try { - producer.close(); - } catch (PulsarClientException e) { - sneakyThrow(e); - } + sneakyClient(producer::close); } } } @@ -575,11 +553,7 @@ private void removeConsumers(String topic) { ConcurrentHashMap> integerConsumers = consumers.remove(topicName); if (integerConsumers != null) { for (Consumer consumer : integerConsumers.values()) { - try { - consumer.close(); - } catch (PulsarClientException e) { - sneakyThrow(e); - } + sneakyClient(consumer::close); } } } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeUtils.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeUtils.java new file mode 100644 index 0000000000000..b064cfa2cd7a1 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeUtils.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.runtime; + +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; + +import java.util.Collections; +import java.util.List; + +import static org.apache.pulsar.common.naming.NamespaceName.SYSTEM_NAMESPACE; +import static org.apache.pulsar.common.naming.TopicName.TRANSACTION_COORDINATOR_ASSIGN; + +/** This class is used to create the basic topics for a standalone Pulsar instance. */ +public final class PulsarRuntimeUtils { + + private PulsarRuntimeUtils() { + // No public constructor + } + + /** Create the system topics. */ + public static void initializePulsarEnvironment( + ServiceConfiguration config, String serviceUrl, String adminUrl) + throws PulsarAdminException, PulsarClientException { + try (PulsarAdmin admin = PulsarAdmin.builder().serviceHttpUrl(adminUrl).build()) { + ClusterData clusterData = + ClusterData.builder().serviceUrl(adminUrl).brokerServiceUrl(serviceUrl).build(); + String cluster = config.getClusterName(); + createSampleNameSpace(admin, clusterData, cluster); + + // Create default namespace + createNameSpace( + admin, + cluster, + TopicName.PUBLIC_TENANT, + TopicName.PUBLIC_TENANT + "/" + TopicName.DEFAULT_NAMESPACE); + + // Create Pulsar system namespace + createNameSpace( + admin, cluster, SYSTEM_NAMESPACE.getTenant(), SYSTEM_NAMESPACE.toString()); + + // Enable transaction + if (config.isTransactionCoordinatorEnabled() + && !admin.namespaces() + .getTopics(SYSTEM_NAMESPACE.toString()) + .contains(TRANSACTION_COORDINATOR_ASSIGN.getPartition(0).toString())) { + admin.topics().createPartitionedTopic(TRANSACTION_COORDINATOR_ASSIGN.toString(), 1); + } + } + } + + private static void createSampleNameSpace( + PulsarAdmin admin, ClusterData clusterData, String cluster) + throws PulsarAdminException { + // Create a sample namespace + String tenant = "sample"; + String globalCluster = "global"; + String namespace = tenant + "/ns1"; + + List clusters = admin.clusters().getClusters(); + if (!clusters.contains(cluster)) { + admin.clusters().createCluster(cluster, clusterData); + } else { + admin.clusters().updateCluster(cluster, clusterData); + } + // Create marker for "global" cluster + if (!clusters.contains(globalCluster)) { + admin.clusters().createCluster(globalCluster, ClusterData.builder().build()); + } + + if (!admin.tenants().getTenants().contains(tenant)) { + admin.tenants() + .createTenant( + tenant, + new TenantInfoImpl( + Collections.emptySet(), Collections.singleton(cluster))); + } + + if (!admin.namespaces().getNamespaces(tenant).contains(namespace)) { + admin.namespaces().createNamespace(namespace); + } + } + + private static void createNameSpace( + PulsarAdmin admin, String cluster, String publicTenant, String defaultNamespace) + throws PulsarAdminException { + if (!admin.tenants().getTenants().contains(publicTenant)) { + admin.tenants() + .createTenant( + publicTenant, + TenantInfo.builder() + .adminRoles(Collections.emptySet()) + .allowedClusters(Collections.singleton(cluster)) + .build()); + } + if (!admin.namespaces().getNamespaces(publicTenant).contains(defaultNamespace)) { + admin.namespaces().createNamespace(defaultNamespace); + admin.namespaces() + .setNamespaceReplicationClusters( + defaultNamespace, Collections.singleton(cluster)); + } + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java index 3d66728fded79..2a2d4d34ecdfc 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java @@ -67,6 +67,10 @@ public class PulsarContainerRuntime implements PulsarRuntime { private boolean boundFlink = false; private PulsarRuntimeOperator operator; + public PulsarContainerRuntime(boolean authEnabled) { + // TODO Add authentication support. + } + public PulsarContainerRuntime bindWithFlinkContainer(GenericContainer flinkContainer) { checkArgument( !started.get(), @@ -90,15 +94,16 @@ public void startUp() { // Override the default configuration in container for enabling the Pulsar transaction. container.withClasspathResourceMapping( - "containers/txnStandalone.conf", - "/pulsar/conf/standalone.conf", - BindMode.READ_ONLY); + "docker/bootstrap.sh", "/pulsar/bin/bootstrap.sh", BindMode.READ_ONLY); // Waiting for the Pulsar border is ready. container.waitingFor( forHttp("/admin/v2/namespaces/public/default") .forPort(BROKER_HTTP_PORT) .forStatusCode(200) .withStartupTimeout(Duration.ofMinutes(5))); + // Set custom startup script. + container.withCommand("sh /pulsar/bin/bootstrap.sh"); + // Start the Pulsar Container. container.start(); // Append the output to this runtime logger. Used for local debug purpose. @@ -124,7 +129,6 @@ public void tearDown() { try { if (operator != null) { operator.close(); - this.operator = null; } container.stop(); started.compareAndSet(true, false); diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/embedded/PulsarEmbeddedRuntime.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/embedded/PulsarEmbeddedRuntime.java deleted file mode 100644 index cf080b8f7a638..0000000000000 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/embedded/PulsarEmbeddedRuntime.java +++ /dev/null @@ -1,273 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.connector.pulsar.testutils.runtime.embedded; - -import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntime; -import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator; -import org.apache.flink.util.FileUtils; - -import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.pulsar.broker.PulsarService; -import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.client.admin.PulsarAdmin; -import org.apache.pulsar.client.admin.PulsarAdminException; -import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; -import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.policies.data.ClusterData; -import org.apache.pulsar.common.policies.data.TenantInfo; -import org.apache.pulsar.common.policies.data.TenantInfoImpl; -import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; -import java.net.URL; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.Collections; -import java.util.List; -import java.util.Optional; - -import static org.apache.flink.util.Preconditions.checkNotNull; -import static org.apache.pulsar.broker.ServiceConfigurationUtils.brokerUrl; -import static org.apache.pulsar.broker.ServiceConfigurationUtils.webServiceUrl; -import static org.apache.pulsar.common.naming.NamespaceName.SYSTEM_NAMESPACE; -import static org.apache.pulsar.common.naming.TopicName.TRANSACTION_COORDINATOR_ASSIGN; - -/** Providing a embedded pulsar server. We use this runtime for transaction related tests. */ -public class PulsarEmbeddedRuntime implements PulsarRuntime { - private static final Logger LOG = LoggerFactory.getLogger(PulsarEmbeddedRuntime.class); - - private static final String CONFIG_FILE_PATH; - - static { - // Find the absolute path for containers/txnStandalone.conf - ClassLoader classLoader = PulsarEmbeddedRuntime.class.getClassLoader(); - URL resource = classLoader.getResource("containers/txnStandalone.conf"); - File file = new File(checkNotNull(resource).getFile()); - CONFIG_FILE_PATH = file.getAbsolutePath(); - } - - private final Path tempDir; - - private LocalBookkeeperEnsemble bookkeeper; - private PulsarService pulsarService; - private PulsarRuntimeOperator operator; - - public PulsarEmbeddedRuntime() { - this.tempDir = createTempDir(); - } - - @Override - public void startUp() { - try { - startBookkeeper(); - startPulsarService(); - - // Create the operator. - this.operator = new PulsarRuntimeOperator(getBrokerUrl(), getWebServiceUrl()); - } catch (Exception e) { - throw new IllegalStateException(e); - } - } - - @Override - public void tearDown() { - try { - if (operator != null) { - operator.close(); - this.operator = null; - } - if (pulsarService != null) { - pulsarService.close(); - } - if (bookkeeper != null) { - bookkeeper.stop(); - } - } catch (Exception e) { - throw new IllegalStateException(e); - } finally { - removeTempDir(tempDir); - } - } - - @Override - public PulsarRuntimeOperator operator() { - return checkNotNull(operator, "You should start this embedded Pulsar first."); - } - - private Path createTempDir() { - try { - return Files.createTempDirectory("pulsar"); - } catch (IOException e) { - throw new IllegalStateException(e); - } - } - - private void removeTempDir(Path tempDir) { - try { - FileUtils.deleteDirectory(tempDir.normalize().toFile()); - } catch (IOException e) { - throw new IllegalStateException(e); - } - } - - public void startBookkeeper() throws Exception { - Path zkPath = Paths.get("data", "standalone", "zookeeper"); - Path bkPath = Paths.get("data", "standalone", "bookkeeper"); - - String zkDir = tempDir.resolve(zkPath).normalize().toString(); - String bkDir = tempDir.resolve(bkPath).normalize().toString(); - - ServerConfiguration bkServerConf = new ServerConfiguration(); - bkServerConf.loadConf(new File(CONFIG_FILE_PATH).toURI().toURL()); - this.bookkeeper = new LocalBookkeeperEnsemble(1, 0, 0, zkDir, bkDir, true, "127.0.0.1"); - - // Start Bookkeeper & zookeeper. - bookkeeper.startStandalone(bkServerConf, false); - } - - private void startPulsarService() throws Exception { - ServiceConfiguration config; - try (FileInputStream inputStream = new FileInputStream(CONFIG_FILE_PATH)) { - config = PulsarConfigurationLoader.create(inputStream, ServiceConfiguration.class); - } catch (IOException e) { - throw new IllegalStateException(e); - } - - // Use runtime dynamic ports for broker. - config.setAdvertisedAddress("127.0.0.1"); - config.setClusterName("standalone"); - - // Use random port. - config.setBrokerServicePort(Optional.of(0)); - config.setWebServicePort(Optional.of(0)); - - // Select available port for bookkeeper and zookeeper. - int zkPort = getZkPort(); - String zkConnect = "127.0.0.1" + ":" + zkPort; - config.setZookeeperServers(zkConnect); - config.setConfigurationStoreServers(zkConnect); - config.setRunningStandalone(true); - - this.pulsarService = new PulsarService(config); - - // Start Pulsar Broker. - pulsarService.start(); - - // Create sample data environment. - String webServiceUrl = getWebServiceUrl(); - String brokerUrl = getBrokerUrl(); - try (PulsarAdmin admin = PulsarAdmin.builder().serviceHttpUrl(webServiceUrl).build()) { - ClusterData clusterData = - ClusterData.builder() - .serviceUrl(webServiceUrl) - .brokerServiceUrl(brokerUrl) - .build(); - String cluster = config.getClusterName(); - createSampleNameSpace(admin, clusterData, cluster); - - // Create default namespace - createNameSpace( - admin, - cluster, - TopicName.PUBLIC_TENANT, - TopicName.PUBLIC_TENANT + "/" + TopicName.DEFAULT_NAMESPACE); - - // Create Pulsar system namespace - createNameSpace( - admin, cluster, SYSTEM_NAMESPACE.getTenant(), SYSTEM_NAMESPACE.toString()); - // Enable transaction - if (config.isTransactionCoordinatorEnabled() - && !admin.namespaces() - .getTopics(SYSTEM_NAMESPACE.toString()) - .contains(TRANSACTION_COORDINATOR_ASSIGN.getPartition(0).toString())) { - admin.topics().createPartitionedTopic(TRANSACTION_COORDINATOR_ASSIGN.toString(), 1); - } - } - } - - private int getZkPort() { - return checkNotNull(bookkeeper).getZookeeperPort(); - } - - private String getBrokerUrl() { - Integer port = pulsarService.getBrokerListenPort().orElseThrow(IllegalStateException::new); - return brokerUrl("127.0.0.1", port); - } - - private String getWebServiceUrl() { - Integer port = pulsarService.getListenPortHTTP().orElseThrow(IllegalArgumentException::new); - return webServiceUrl("127.0.0.1", port); - } - - private void createSampleNameSpace(PulsarAdmin admin, ClusterData clusterData, String cluster) - throws PulsarAdminException { - // Create a sample namespace - String tenant = "sample"; - String globalCluster = "global"; - String namespace = tenant + "/ns1"; - - List clusters = admin.clusters().getClusters(); - if (!clusters.contains(cluster)) { - admin.clusters().createCluster(cluster, clusterData); - } else { - admin.clusters().updateCluster(cluster, clusterData); - } - // Create marker for "global" cluster - if (!clusters.contains(globalCluster)) { - admin.clusters().createCluster(globalCluster, ClusterData.builder().build()); - } - - if (!admin.tenants().getTenants().contains(tenant)) { - admin.tenants() - .createTenant( - tenant, - new TenantInfoImpl( - Collections.emptySet(), Collections.singleton(cluster))); - } - - if (!admin.namespaces().getNamespaces(tenant).contains(namespace)) { - admin.namespaces().createNamespace(namespace); - } - } - - private void createNameSpace( - PulsarAdmin admin, String cluster, String publicTenant, String defaultNamespace) - throws PulsarAdminException { - if (!admin.tenants().getTenants().contains(publicTenant)) { - admin.tenants() - .createTenant( - publicTenant, - TenantInfo.builder() - .adminRoles(Collections.emptySet()) - .allowedClusters(Collections.singleton(cluster)) - .build()); - } - if (!admin.namespaces().getNamespaces(publicTenant).contains(defaultNamespace)) { - admin.namespaces().createNamespace(defaultNamespace); - admin.namespaces() - .setNamespaceReplicationClusters( - defaultNamespace, Collections.singleton(cluster)); - } - } -} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/BlankBrokerInterceptor.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/BlankBrokerInterceptor.java new file mode 100644 index 0000000000000..8355a23279f09 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/BlankBrokerInterceptor.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.runtime.mock; + +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.intercept.BrokerInterceptor; +import org.apache.pulsar.broker.service.ServerCnx; +import org.apache.pulsar.common.api.proto.BaseCommand; + +import javax.servlet.ServletRequest; +import javax.servlet.ServletResponse; + +/** No operation for this BrokerInterceptor implementation. */ +public class BlankBrokerInterceptor implements BrokerInterceptor { + + @Override + public void onPulsarCommand(BaseCommand command, ServerCnx cnx) { + // no-op + } + + @Override + public void onConnectionClosed(ServerCnx cnx) { + // no-op + } + + @Override + public void onWebserviceRequest(ServletRequest request) { + // no-op + } + + @Override + public void onWebserviceResponse(ServletRequest request, ServletResponse response) { + // no-op + } + + @Override + public void initialize(PulsarService pulsarService) { + // no-op + } + + @Override + public void close() { + // no-op + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/MockBookKeeperClientFactory.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/MockBookKeeperClientFactory.java new file mode 100644 index 0000000000000..d35bd669ab045 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/MockBookKeeperClientFactory.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.runtime.mock; + +import io.netty.channel.EventLoopGroup; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.EnsemblePlacementPolicy; +import org.apache.bookkeeper.common.util.OrderedExecutor; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.pulsar.broker.BookKeeperClientFactory; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; + +import java.io.IOException; +import java.util.Map; +import java.util.Optional; + +/** A BookKeeperClientFactory implementation which returns a mocked bookkeeper. */ +public class MockBookKeeperClientFactory implements BookKeeperClientFactory { + + private final OrderedExecutor executor = + OrderedExecutor.newBuilder().numThreads(1).name("mock-pulsar-bookkeeper").build(); + + private final BookKeeper bookKeeper = NonClosableMockBookKeeper.create(executor); + + @Override + public BookKeeper create( + ServiceConfiguration conf, + MetadataStoreExtended store, + EventLoopGroup eventLoopGroup, + Optional> ensemblePlacementPolicyClass, + Map ensemblePlacementPolicyProperties) + throws IOException { + return bookKeeper; + } + + @Override + public BookKeeper create( + ServiceConfiguration conf, + MetadataStoreExtended store, + EventLoopGroup eventLoopGroup, + Optional> ensemblePlacementPolicyClass, + Map ensemblePlacementPolicyProperties, + StatsLogger statsLogger) + throws IOException { + return bookKeeper; + } + + @Override + public void close() { + try { + bookKeeper.close(); + executor.shutdown(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/MockPulsarService.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/MockPulsarService.java new file mode 100644 index 0000000000000..09015a7db8660 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/MockPulsarService.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.runtime.mock; + +import org.apache.bookkeeper.common.util.OrderedExecutor; +import org.apache.pulsar.broker.BookKeeperClientFactory; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.intercept.BrokerInterceptor; +import org.apache.pulsar.broker.namespace.NamespaceService; + +import java.util.function.Supplier; + +/** A Mock pulsar service which would use the mocked zookeeper and bookkeeper. */ +public class MockPulsarService extends PulsarService { + + private final SameThreadOrderedSafeExecutor orderedExecutor = + new SameThreadOrderedSafeExecutor(); + + public MockPulsarService(ServiceConfiguration config) { + super(config); + } + + public BookKeeperClientFactory newBookKeeperClientFactory() { + return new MockBookKeeperClientFactory(); + } + + public Supplier getNamespaceServiceProvider() { + return () -> new NamespaceService(this); + } + + @Override + public OrderedExecutor getOrderedExecutor() { + return orderedExecutor; + } + + @Override + public BrokerInterceptor getBrokerInterceptor() { + return new BlankBrokerInterceptor(); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/NonClosableMockBookKeeper.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/NonClosableMockBookKeeper.java new file mode 100644 index 0000000000000..b7001b8ca82f4 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/NonClosableMockBookKeeper.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.runtime.mock; + +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.PulsarMockBookKeeper; +import org.apache.bookkeeper.common.util.OrderedExecutor; + +/** + * Prevent the MockBookKeeper instance from being closed when the broker is restarted within a test. + */ +public class NonClosableMockBookKeeper extends PulsarMockBookKeeper { + + private NonClosableMockBookKeeper(OrderedExecutor executor) throws Exception { + super(executor); + } + + @Override + public void close() { + // no-op + } + + @Override + public void shutdown() { + // no-op + } + + public void reallyShutdown() { + super.shutdown(); + } + + public static BookKeeper create(OrderedExecutor executor) { + try { + return new NonClosableMockBookKeeper(executor); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/PulsarMockRuntime.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/PulsarMockRuntime.java new file mode 100644 index 0000000000000..a86ff5283f558 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/PulsarMockRuntime.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.runtime.mock; + +import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntime; +import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator; + +import org.apache.pulsar.broker.ServiceConfiguration; + +import java.util.Optional; + +import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; +import static org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeUtils.initializePulsarEnvironment; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Providing a mocked pulsar server. */ +public class PulsarMockRuntime implements PulsarRuntime { + + private static final String CLUSTER_NAME = "mock-pulsar-" + randomAlphanumeric(6); + private final ServiceConfiguration configuration; + private final MockPulsarService pulsarService; + private PulsarRuntimeOperator operator; + + public PulsarMockRuntime() { + this(createConfig()); + } + + public PulsarMockRuntime(ServiceConfiguration configuration) { + this.configuration = configuration; + this.pulsarService = new MockPulsarService(configuration); + } + + @Override + public void startUp() { + try { + pulsarService.start(); + + String serviceUrl = pulsarService.getBrokerServiceUrl(); + String adminUrl = pulsarService.getWebServiceAddress(); + initializePulsarEnvironment(configuration, serviceUrl, adminUrl); + + this.operator = new PulsarRuntimeOperator(serviceUrl, adminUrl); + } catch (Exception e) { + throw new IllegalStateException(e); + } + } + + @Override + public void tearDown() { + try { + pulsarService.close(); + operator.close(); + this.operator = null; + } catch (Exception e) { + throw new IllegalStateException(e); + } + } + + @Override + public PulsarRuntimeOperator operator() { + return checkNotNull(operator, "You should start this mock pulsar first."); + } + + private static ServiceConfiguration createConfig() { + ServiceConfiguration configuration = new ServiceConfiguration(); + + configuration.setAdvertisedAddress("localhost"); + configuration.setClusterName(CLUSTER_NAME); + + configuration.setManagedLedgerCacheSizeMB(8); + configuration.setActiveConsumerFailoverDelayTimeMillis(0); + configuration.setDefaultRetentionTimeInMinutes(7); + configuration.setDefaultNumberOfNamespaceBundles(1); + configuration.setMetadataStoreUrl("memory:local"); + configuration.setConfigurationMetadataStoreUrl("memory:local"); + + configuration.setAuthenticationEnabled(false); + configuration.setAuthorizationEnabled(false); + configuration.setAllowAutoTopicCreation(true); + configuration.setBrokerDeleteInactiveTopicsEnabled(false); + + configuration.setWebSocketServiceEnabled(false); + // Use runtime dynamic ports + configuration.setBrokerServicePort(Optional.of(0)); + configuration.setWebServicePort(Optional.of(0)); + + // Enable transactions. + configuration.setSystemTopicEnabled(true); + configuration.setBrokerDeduplicationEnabled(true); + configuration.setTransactionCoordinatorEnabled(true); + configuration.setTransactionMetadataStoreProviderClassName( + "org.apache.pulsar.transaction.coordinator.impl.MLTransactionMetadataStoreProvider"); + + return configuration; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/SameThreadOrderedSafeExecutor.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/SameThreadOrderedSafeExecutor.java new file mode 100644 index 0000000000000..9667f08414f70 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/SameThreadOrderedSafeExecutor.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.runtime.mock; + +import org.apache.bookkeeper.common.util.OrderedExecutor; +import org.apache.bookkeeper.common.util.SafeRunnable; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.pulsar.shade.io.netty.util.concurrent.DefaultThreadFactory; + +/** Override the default bookkeeper executor for executing in one thread executor. */ +public class SameThreadOrderedSafeExecutor extends OrderedExecutor { + + public SameThreadOrderedSafeExecutor() { + super( + "same-thread-executor", + 1, + new DefaultThreadFactory("test"), + NullStatsLogger.INSTANCE, + false, + false, + 100000, + -1, + false); + } + + @Override + public void execute(Runnable r) { + r.run(); + } + + @Override + public void executeOrdered(int orderingKey, SafeRunnable r) { + r.run(); + } + + @Override + public void executeOrdered(long orderingKey, SafeRunnable r) { + r.run(); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarPartitionDataReader.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarPartitionDataReader.java new file mode 100644 index 0000000000000..6ef9510d11683 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarPartitionDataReader.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.sink; + +import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator; +import org.apache.flink.connector.testframe.external.ExternalSystemDataReader; + +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.client.api.SubscriptionMode; +import org.apache.pulsar.client.api.SubscriptionType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; + +/** The data reader for a specified topic partition from Pulsar. */ +public class PulsarPartitionDataReader implements ExternalSystemDataReader, Closeable { + + private static final Logger LOG = LoggerFactory.getLogger(PulsarPartitionDataReader.class); + + private final Consumer consumer; + + public PulsarPartitionDataReader( + PulsarRuntimeOperator operator, String fullTopicName, Schema schema) + throws PulsarClientException { + // Create client for supporting the use in E2E test. + String subscriptionName = randomAlphanumeric(12); + this.consumer = + operator.client() + .newConsumer(schema) + .topic(fullTopicName) + .subscriptionName(subscriptionName) + .subscriptionType(SubscriptionType.Exclusive) + .subscriptionMode(SubscriptionMode.Durable) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + } + + @Override + public List poll(Duration timeout) { + List results = new ArrayList<>(); + + while (true) { + try { + int millis = Math.toIntExact(timeout.toMillis()); + Message message = consumer.receive(millis, MILLISECONDS); + + if (message != null) { + consumer.acknowledgeCumulative(message); + results.add(message.getValue()); + } else { + break; + } + } catch (Exception e) { + LOG.error("", e); + break; + } + } + + return results; + } + + @Override + public void close() throws PulsarClientException { + consumer.close(); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarSinkTestContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarSinkTestContext.java new file mode 100644 index 0000000000000..af00b16ddf0b1 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarSinkTestContext.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.sink; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.pulsar.sink.PulsarSink; +import org.apache.flink.connector.pulsar.testutils.PulsarTestContext; +import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; +import org.apache.flink.connector.testframe.external.ExternalSystemDataReader; +import org.apache.flink.connector.testframe.external.sink.DataStreamSinkV2ExternalContext; +import org.apache.flink.connector.testframe.external.sink.TestingSinkSettings; + +import org.apache.flink.shaded.guava30.com.google.common.io.Closer; + +import org.apache.pulsar.client.api.Schema; + +import java.util.ArrayList; +import java.util.List; +import java.util.Random; + +import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; +import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; +import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_BATCHING_MAX_MESSAGES; +import static org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema.pulsarSchema; +import static org.apache.flink.connector.pulsar.testutils.PulsarTestCommonUtils.toDeliveryGuarantee; + +/** Common sink test context for pulsar based test. */ +public class PulsarSinkTestContext extends PulsarTestContext + implements DataStreamSinkV2ExternalContext { + + private static final String TOPIC_NAME_PREFIX = "flink-sink-topic-"; + private static final int RECORD_SIZE_UPPER_BOUND = 300; + private static final int RECORD_SIZE_LOWER_BOUND = 100; + private static final int RECORD_STRING_SIZE = 20; + + private String topicName = topicName(); + private final Closer closer = Closer.create(); + + public PulsarSinkTestContext(PulsarTestEnvironment environment) { + super(environment, Schema.STRING); + } + + @Override + protected String displayName() { + return "write messages into one topic in Pulsar"; + } + + @Override + public Sink createSink(TestingSinkSettings sinkSettings) { + if (!operator.topicExists(topicName)) { + operator.createTopic(topicName, 4); + } + DeliveryGuarantee guarantee = toDeliveryGuarantee(sinkSettings.getCheckpointingMode()); + + return PulsarSink.builder() + .setServiceUrl(operator.serviceUrl()) + .setAdminUrl(operator.adminUrl()) + .setTopics(topicName) + .setDeliveryGuarantee(guarantee) + .setSerializationSchema(pulsarSchema(Schema.STRING)) + .enableSchemaEvolution() + .setConfig(PULSAR_BATCHING_MAX_MESSAGES, 4) + .build(); + } + + @Override + public ExternalSystemDataReader createSinkDataReader(TestingSinkSettings sinkSettings) { + PulsarPartitionDataReader reader = + sneakyClient( + () -> new PulsarPartitionDataReader<>(operator, topicName, Schema.STRING)); + closer.register(reader); + + return reader; + } + + @Override + public List generateTestData(TestingSinkSettings sinkSettings, long seed) { + Random random = new Random(seed); + int recordSize = + random.nextInt(RECORD_SIZE_UPPER_BOUND - RECORD_SIZE_LOWER_BOUND) + + RECORD_SIZE_LOWER_BOUND; + List records = new ArrayList<>(recordSize); + for (int i = 0; i < recordSize; i++) { + int size = random.nextInt(RECORD_STRING_SIZE) + RECORD_STRING_SIZE; + String record = "index:" + i + "-data:" + randomAlphanumeric(size); + records.add(record); + } + + return records; + } + + @Override + public TypeInformation getProducedType() { + return Types.STRING; + } + + @Override + public void close() throws Exception { + // Change the topic name after finishing a test case. + closer.register(() -> topicName = topicName()); + closer.close(); + } + + private String topicName() { + return TOPIC_NAME_PREFIX + randomAlphanumeric(8); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarSinkTestSuiteBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarSinkTestSuiteBase.java new file mode 100644 index 0000000000000..0695a43569b68 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarSinkTestSuiteBase.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.sink; + +import org.apache.flink.connector.testframe.environment.TestEnvironment; +import org.apache.flink.connector.testframe.external.sink.DataStreamSinkExternalContext; +import org.apache.flink.connector.testframe.testsuites.SinkTestSuiteBase; +import org.apache.flink.streaming.api.CheckpointingMode; + +import org.junit.jupiter.api.Disabled; + +/** Pulsar sink don't expose the monitor metrics now. We have to disable this test. */ +public abstract class PulsarSinkTestSuiteBase extends SinkTestSuiteBase { + + @Override + @Disabled("Enable this test after FLINK-26027 being merged.") + public void testMetrics( + TestEnvironment testEnv, + DataStreamSinkExternalContext externalContext, + CheckpointingMode semantic) {} +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarPartitionDataWriter.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/PulsarPartitionDataWriter.java similarity index 69% rename from flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarPartitionDataWriter.java rename to flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/PulsarPartitionDataWriter.java index da48e26cd2589..1ceb2922bc396 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarPartitionDataWriter.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/PulsarPartitionDataWriter.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.connector.pulsar.testutils; +package org.apache.flink.connector.pulsar.testutils.source; import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator; import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter; @@ -25,20 +25,26 @@ import java.util.List; -/** Source split data writer for writing test data into a Pulsar topic partition. */ -public class PulsarPartitionDataWriter implements ExternalSystemSplitDataWriter { +/** + * Source split data writer for writing test data into a Pulsar topic partition. This writer doesn't + * need to be closed. + */ +public class PulsarPartitionDataWriter implements ExternalSystemSplitDataWriter { private final PulsarRuntimeOperator operator; private final String fullTopicName; + private final Schema schema; - public PulsarPartitionDataWriter(PulsarRuntimeOperator operator, String fullTopicName) { + public PulsarPartitionDataWriter( + PulsarRuntimeOperator operator, String fullTopicName, Schema schema) { this.operator = operator; this.fullTopicName = fullTopicName; + this.schema = schema; } @Override - public void writeRecords(List records) { - operator.sendMessages(fullTopicName, Schema.STRING, records); + public void writeRecords(List records) { + operator.sendMessages(fullTopicName, schema, records); } @Override diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/PulsarSourceTestContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/PulsarSourceTestContext.java new file mode 100644 index 0000000000000..8089f8c58e734 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/PulsarSourceTestContext.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.source; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.connector.pulsar.source.PulsarSource; +import org.apache.flink.connector.pulsar.source.PulsarSourceBuilder; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.testutils.PulsarTestContext; +import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; +import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter; +import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext; +import org.apache.flink.connector.testframe.external.source.TestingSourceSettings; + +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; + +import java.util.List; +import java.util.Random; +import java.util.stream.IntStream; + +import static java.util.stream.Collectors.toList; +import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS; +import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.pulsarSchema; +import static org.apache.pulsar.client.api.RegexSubscriptionMode.AllTopics; + +/** + * Common source test context for pulsar based test. We use the string text as the basic send + * content. + */ +public abstract class PulsarSourceTestContext extends PulsarTestContext + implements DataStreamSourceExternalContext { + + private static final long DISCOVERY_INTERVAL = 1000L; + private static final int BATCH_DATA_SIZE = 300; + + protected PulsarSourceTestContext(PulsarTestEnvironment environment) { + super(environment, Schema.STRING); + } + + @Override + public Source createSource(TestingSourceSettings sourceSettings) { + PulsarSourceBuilder builder = + PulsarSource.builder() + .setDeserializationSchema(pulsarSchema(schema)) + .setServiceUrl(operator.serviceUrl()) + .setAdminUrl(operator.adminUrl()) + .setTopicPattern(topicPattern(), AllTopics) + .setSubscriptionType(subscriptionType()) + .setSubscriptionName(subscriptionName()) + .setConfig(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS, DISCOVERY_INTERVAL); + + // Set extra configuration for source builder. + setSourceBuilder(builder); + + if (sourceSettings.getBoundedness() == Boundedness.BOUNDED) { + // Using the latest stop cursor for making sure the source could be stopped. + // This is required for SourceTestSuiteBase. + builder.setBoundedStopCursor(StopCursor.latest()); + } + + return builder.build(); + } + + @Override + public ExternalSystemSplitDataWriter createSourceSplitDataWriter( + TestingSourceSettings sourceSettings) { + String partitionName = generatePartitionName(); + return new PulsarPartitionDataWriter<>(operator, partitionName, schema); + } + + @Override + public List generateTestData( + TestingSourceSettings sourceSettings, int splitIndex, long seed) { + Random random = new Random(seed); + return IntStream.range(0, BATCH_DATA_SIZE) + .boxed() + .map( + index -> { + int length = random.nextInt(20) + 1; + return "split:" + + splitIndex + + "-index:" + + index + + "-content:" + + randomAlphanumeric(length); + }) + .collect(toList()); + } + + @Override + public TypeInformation getProducedType() { + return Types.STRING; + } + + /** Override this method for creating builder. */ + protected void setSourceBuilder(PulsarSourceBuilder builder) { + // Nothing to do by default. + } + + /** + * The topic pattern which is used in Pulsar topic auto discovery. It was discovered every + * {@link #DISCOVERY_INTERVAL} ms; + */ + protected abstract String topicPattern(); + + /** The subscription name used in Pulsar consumer. */ + protected abstract String subscriptionName(); + + /** The subscription type used in Pulsar consumer. */ + protected abstract SubscriptionType subscriptionType(); + + /** + * Dynamic generate a partition related topic in Pulsar. This topic should be pre-created in + * Pulsar. Everytime we call this method, we may get a new partition name. + */ + protected abstract String generatePartitionName(); +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/MultipleTopicConsumingContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/MultipleTopicConsumingContext.java new file mode 100644 index 0000000000000..d89e153f9f17b --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/MultipleTopicConsumingContext.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.source.cases; + +import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; +import org.apache.flink.connector.pulsar.testutils.source.PulsarSourceTestContext; + +import org.apache.pulsar.client.api.SubscriptionType; + +import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; + +/** + * Pulsar external context that will create multiple topics with only one partitions as source + * splits. + */ +public class MultipleTopicConsumingContext extends PulsarSourceTestContext { + + private final String topicPrefix = "flink-multiple-topic-" + randomAlphabetic(8) + "-"; + + private int index = 0; + + public MultipleTopicConsumingContext(PulsarTestEnvironment environment) { + super(environment); + } + + @Override + protected String displayName() { + return "consume message on multiple topic"; + } + + @Override + protected String topicPattern() { + return topicPrefix + ".+"; + } + + @Override + protected String subscriptionName() { + return "flink-multiple-topic-test"; + } + + @Override + protected SubscriptionType subscriptionType() { + return SubscriptionType.Exclusive; + } + + @Override + protected String generatePartitionName() { + String topic = topicPrefix + index; + operator.createTopic(topic, 1); + registerTopic(topic); + index++; + + return topicNameWithPartition(topic, 0); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/SingleTopicConsumingContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/SingleTopicConsumingContext.java new file mode 100644 index 0000000000000..80e1ff16bff44 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/SingleTopicConsumingContext.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.source.cases; + +import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; +import org.apache.flink.connector.pulsar.testutils.source.PulsarSourceTestContext; + +import org.apache.pulsar.client.api.SubscriptionType; + +import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; + +/** + * A Pulsar external context that will create only one topic and use partitions in that topic as + * source splits. + */ +public class SingleTopicConsumingContext extends PulsarSourceTestContext { + + private final String topicName = "pulsar-single-topic-" + randomAlphanumeric(8); + + private int index = 0; + + public SingleTopicConsumingContext(PulsarTestEnvironment environment) { + super(environment); + registerTopic(topicName); + } + + @Override + protected String displayName() { + return "consume message on single topic"; + } + + @Override + protected String topicPattern() { + return topicName + ".+"; + } + + @Override + protected String subscriptionName() { + return "pulsar-single-topic-test"; + } + + @Override + protected SubscriptionType subscriptionType() { + return SubscriptionType.Exclusive; + } + + @Override + protected String generatePartitionName() { + if (index == 0) { + operator.createTopic(topicName, index + 1); + } else { + operator.increaseTopicPartitions(topicName, index + 1); + } + + return topicNameWithPartition(topicName, index++); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/resources/containers/txnStandalone.conf b/flink-connectors/flink-connector-pulsar/src/test/resources/containers/txnStandalone.conf deleted file mode 100644 index bf35c59860b23..0000000000000 --- a/flink-connectors/flink-connector-pulsar/src/test/resources/containers/txnStandalone.conf +++ /dev/null @@ -1,1042 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -# - -### --- General broker settings --- ### - -# Zookeeper quorum connection string -zookeeperServers= - -# Configuration Store connection string -configurationStoreServers= - -brokerServicePort=6650 - -# Port to use to server HTTP request -webServicePort=8080 - -# Hostname or IP address the service binds on, default is 0.0.0.0. -bindAddress=0.0.0.0 - -# Extra bind addresses for the service: :://:,[...] -bindAddresses= - -# Hostname or IP address the service advertises to the outside world. If not set, the value of InetAddress.getLocalHost().getHostName() is used. -advertisedAddress= - -# Enable or disable the HAProxy protocol. -haProxyProtocolEnabled=false - -# Number of threads to use for Netty IO. Default is set to 2 * Runtime.getRuntime().availableProcessors() -numIOThreads= - -# Number of threads to use for ordered executor. The ordered executor is used to operate with zookeeper, -# such as init zookeeper client, get namespace policies from zookeeper etc. It also used to split bundle. Default is 8 -numOrderedExecutorThreads=8 - -# Number of threads to use for HTTP requests processing. Default is set to 2 * Runtime.getRuntime().availableProcessors() -numHttpServerThreads= - -# Number of thread pool size to use for pulsar broker service. -# The executor in thread pool will do basic broker operation like load/unload bundle, update managedLedgerConfig, -# update topic/subscription/replicator message dispatch rate, do leader election etc. -# Default is Runtime.getRuntime().availableProcessors() -numExecutorThreadPoolSize= - -# Number of thread pool size to use for pulsar zookeeper callback service -# The cache executor thread pool is used for restarting global zookeeper session. -# Default is 10 -numCacheExecutorThreadPoolSize=10 - -# Max concurrent web requests -maxConcurrentHttpRequests=1024 - -# Name of the cluster to which this broker belongs to -clusterName=standalone - -# Enable cluster's failure-domain which can distribute brokers into logical region -failureDomainsEnabled=false - -# Zookeeper session timeout in milliseconds -zooKeeperSessionTimeoutMillis=30000 - -# ZooKeeper operation timeout in seconds -zooKeeperOperationTimeoutSeconds=30 - -# ZooKeeper cache expiry time in seconds -zooKeeperCacheExpirySeconds=300 - -# Time to wait for broker graceful shutdown. After this time elapses, the process will be killed -brokerShutdownTimeoutMs=60000 - -# Flag to skip broker shutdown when broker handles Out of memory error -skipBrokerShutdownOnOOM=false - -# Enable backlog quota check. Enforces action on topic when the quota is reached -backlogQuotaCheckEnabled=true - -# How often to check for topics that have reached the quota -backlogQuotaCheckIntervalInSeconds=60 - -# Default per-topic backlog quota limit -backlogQuotaDefaultLimitGB=10 - -# Default per-topic backlog quota time limit in second, less than 0 means no limitation. default is -1. -backlogQuotaDefaultLimitSecond=-1 - -# Default ttl for namespaces if ttl is not already configured at namespace policies. (disable default-ttl with value 0) -ttlDurationDefaultInSeconds=0 - -# Enable the deletion of inactive topics. This parameter need to cooperate with the allowAutoTopicCreation parameter. -# If brokerDeleteInactiveTopicsEnabled is set to true, we should ensure that allowAutoTopicCreation is also set to true. -brokerDeleteInactiveTopicsEnabled=true - -# How often to check for inactive topics -brokerDeleteInactiveTopicsFrequencySeconds=60 - -# Allow you to delete a tenant forcefully. -forceDeleteTenantAllowed=false - -# Allow you to delete a namespace forcefully. -forceDeleteNamespaceAllowed=false - -# Max pending publish requests per connection to avoid keeping large number of pending -# requests in memory. Default: 1000 -maxPendingPublishRequestsPerConnection=1000 - -# How frequently to proactively check and purge expired messages -messageExpiryCheckIntervalInMinutes=5 - -# Check between intervals to see if max message size in topic policies has been updated. -# Default is 60s -maxMessageSizeCheckIntervalInSeconds=60 - -# How long to delay rewinding cursor and dispatching messages when active consumer is changed -activeConsumerFailoverDelayTimeMillis=1000 - -# How long to delete inactive subscriptions from last consuming -# When it is 0, inactive subscriptions are not deleted automatically -subscriptionExpirationTimeMinutes=0 - -# Enable subscription message redelivery tracker to send redelivery count to consumer (default is enabled) -subscriptionRedeliveryTrackerEnabled=true - -# On KeyShared subscriptions, with default AUTO_SPLIT mode, use splitting ranges or -# consistent hashing to reassign keys to new consumers -subscriptionKeySharedUseConsistentHashing=false - -# On KeyShared subscriptions, number of points in the consistent-hashing ring. -# The higher the number, the more equal the assignment of keys to consumers -subscriptionKeySharedConsistentHashingReplicaPoints=100 - -# How frequently to proactively check and purge expired subscription -subscriptionExpiryCheckIntervalInMinutes=5 - -# Set the default behavior for message deduplication in the broker -# This can be overridden per-namespace. If enabled, broker will reject -# messages that were already stored in the topic -brokerDeduplicationEnabled=false - -# Maximum number of producer information that it's going to be -# persisted for deduplication purposes -brokerDeduplicationMaxNumberOfProducers=10000 - -# Number of entries after which a dedup info snapshot is taken. -# A bigger interval will lead to less snapshots being taken though it would -# increase the topic recovery time, when the entries published after the -# snapshot need to be replayed -brokerDeduplicationEntriesInterval=1000 - -# Time of inactivity after which the broker will discard the deduplication information -# relative to a disconnected producer. Default is 6 hours. -brokerDeduplicationProducerInactivityTimeoutMinutes=360 - -# When a namespace is created without specifying the number of bundle, this -# value will be used as the default -defaultNumberOfNamespaceBundles=4 - -# Max number of topics allowed to be created in the namespace. When the topics reach the max topics of the namespace, -# the broker should reject the new topic request(include topic auto-created by the producer or consumer) -# until the number of connected consumers decrease. -# Using a value of 0, is disabling maxTopicsPerNamespace-limit check. -maxTopicsPerNamespace=0 - -# Allow schema to be auto updated at broker level. User can override this by -# 'is_allow_auto_update_schema' of namespace policy. -isAllowAutoUpdateSchemaEnabled=true - -# Enable check for minimum allowed client library version -clientLibraryVersionCheckEnabled=false - -# Path for the file used to determine the rotation status for the broker when responding -# to service discovery health checks -statusFilePath=/usr/local/apache/htdocs - -# Max number of unacknowledged messages allowed to receive messages by a consumer on a shared subscription. Broker will stop sending -# messages to consumer once, this limit reaches until consumer starts acknowledging messages back -# Using a value of 0, is disabling unackeMessage limit check and consumer can receive messages without any restriction -maxUnackedMessagesPerConsumer=50000 - -# Max number of unacknowledged messages allowed per shared subscription. Broker will stop dispatching messages to -# all consumers of the subscription once this limit reaches until consumer starts acknowledging messages back and -# unack count reaches to limit/2. Using a value of 0, is disabling unackedMessage-limit -# check and dispatcher can dispatch messages without any restriction -maxUnackedMessagesPerSubscription=200000 - -# Max number of unacknowledged messages allowed per broker. Once this limit reaches, broker will stop dispatching -# messages to all shared subscription which has higher number of unack messages until subscriptions start -# acknowledging messages back and unack count reaches to limit/2. Using a value of 0, is disabling -# unackedMessage-limit check and broker doesn't block dispatchers -maxUnackedMessagesPerBroker=0 - -# Once broker reaches maxUnackedMessagesPerBroker limit, it blocks subscriptions which has higher unacked messages -# than this percentage limit and subscription will not receive any new messages until that subscription acks back -# limit/2 messages -maxUnackedMessagesPerSubscriptionOnBrokerBlocked=0.16 - -# Tick time to schedule task that checks topic publish rate limiting across all topics -# Reducing to lower value can give more accuracy while throttling publish but -# it uses more CPU to perform frequent check. (Disable publish throttling with value 0) -topicPublisherThrottlingTickTimeMillis=2 - -# Enable precise rate limit for topic publish -preciseTopicPublishRateLimiterEnable=false - -# Tick time to schedule task that checks broker publish rate limiting across all topics -# Reducing to lower value can give more accuracy while throttling publish but -# it uses more CPU to perform frequent check. (Disable publish throttling with value 0) -brokerPublisherThrottlingTickTimeMillis=50 - -# Max Rate(in 1 seconds) of Message allowed to publish for a broker if broker publish rate limiting enabled -# (Disable message rate limit with value 0) -brokerPublisherThrottlingMaxMessageRate=0 - -# Max Rate(in 1 seconds) of Byte allowed to publish for a broker if broker publish rate limiting enabled -# (Disable byte rate limit with value 0) -brokerPublisherThrottlingMaxByteRate=0 - -# Default messages per second dispatch throttling-limit for every topic. Using a value of 0, is disabling default -# message dispatch-throttling -dispatchThrottlingRatePerTopicInMsg=0 - -# Default bytes per second dispatch throttling-limit for every topic. Using a value of 0, is disabling -# default message-byte dispatch-throttling -dispatchThrottlingRatePerTopicInByte=0 - -# Apply dispatch rate limiting on batch message instead individual -# messages with in batch message. (Default is disabled) -dispatchThrottlingOnBatchMessageEnabled=false - -# Dispatch rate-limiting relative to publish rate. -# (Enabling flag will make broker to dynamically update dispatch-rate relatively to publish-rate: -# throttle-dispatch-rate = (publish-rate + configured dispatch-rate). -dispatchThrottlingRateRelativeToPublishRate=false - -# By default we enable dispatch-throttling for both caught up consumers as well as consumers who have -# backlog. -dispatchThrottlingOnNonBacklogConsumerEnabled=true - -# The read failure backoff initial time in milliseconds. By default it is 15s. -dispatcherReadFailureBackoffInitialTimeInMs=15000 - -# The read failure backoff max time in milliseconds. By default it is 60s. -dispatcherReadFailureBackoffMaxTimeInMs=60000 - -# The read failure backoff mandatory stop time in milliseconds. By default it is 0s. -dispatcherReadFailureBackoffMandatoryStopTimeInMs=0 - -# Precise dispathcer flow control according to history message number of each entry -preciseDispatcherFlowControl=false - -# Max number of concurrent lookup request broker allows to throttle heavy incoming lookup traffic -maxConcurrentLookupRequest=50000 - -# Max number of concurrent topic loading request broker allows to control number of zk-operations -maxConcurrentTopicLoadRequest=5000 - -# Max concurrent non-persistent message can be processed per connection -maxConcurrentNonPersistentMessagePerConnection=1000 - -# Number of worker threads to serve non-persistent topic -numWorkerThreadsForNonPersistentTopic=8 - -# Enable broker to load persistent topics -enablePersistentTopics=true - -# Enable broker to load non-persistent topics -enableNonPersistentTopics=true - -# Max number of producers allowed to connect to topic. Once this limit reaches, Broker will reject new producers -# until the number of connected producers decrease. -# Using a value of 0, is disabling maxProducersPerTopic-limit check. -maxProducersPerTopic=0 - -# Max number of producers with the same IP address allowed to connect to topic. -# Once this limit reaches, Broker will reject new producers until the number of -# connected producers with the same IP address decrease. -# Using a value of 0, is disabling maxSameAddressProducersPerTopic-limit check. -maxSameAddressProducersPerTopic=0 - -# Enforce producer to publish encrypted messages.(default disable). -encryptionRequireOnProducer=false - -# Max number of consumers allowed to connect to topic. Once this limit reaches, Broker will reject new consumers -# until the number of connected consumers decrease. -# Using a value of 0, is disabling maxConsumersPerTopic-limit check. -maxConsumersPerTopic=0 - -# Max number of consumers with the same IP address allowed to connect to topic. -# Once this limit reaches, Broker will reject new consumers until the number of -# connected consumers with the same IP address decrease. -# Using a value of 0, is disabling maxSameAddressConsumersPerTopic-limit check. -maxSameAddressConsumersPerTopic=0 - -# Max number of subscriptions allowed to subscribe to topic. Once this limit reaches, broker will reject -# new subscription until the number of subscribed subscriptions decrease. -# Using a value of 0, is disabling maxSubscriptionsPerTopic limit check. -maxSubscriptionsPerTopic=0 - -# Max number of consumers allowed to connect to subscription. Once this limit reaches, Broker will reject new consumers -# until the number of connected consumers decrease. -# Using a value of 0, is disabling maxConsumersPerSubscription-limit check. -maxConsumersPerSubscription=0 - -# Max number of partitions per partitioned topic -# Use 0 or negative number to disable the check -maxNumPartitionsPerPartitionedTopic=0 - -### --- Metadata Store --- ### - -# Whether we should enable metadata operations batching -metadataStoreBatchingEnabled=true - -# Maximum delay to impose on batching grouping -metadataStoreBatchingMaxDelayMillis=5 - -# Maximum number of operations to include in a singular batch -metadataStoreBatchingMaxOperations=1000 - -# Maximum size of a batch -metadataStoreBatchingMaxSizeKb=128 - -### --- TLS --- ### -# Deprecated - Use webServicePortTls and brokerServicePortTls instead -tlsEnabled=false - -# Tls cert refresh duration in seconds (set 0 to check on every new connection) -tlsCertRefreshCheckDurationSec=300 - -# Path for the TLS certificate file -tlsCertificateFilePath= - -# Path for the TLS private key file -tlsKeyFilePath= - -# Path for the trusted TLS certificate file. -# This cert is used to verify that any certs presented by connecting clients -# are signed by a certificate authority. If this verification -# fails, then the certs are untrusted and the connections are dropped. -tlsTrustCertsFilePath= - -# Accept untrusted TLS certificate from client. -# If true, a client with a cert which cannot be verified with the -# 'tlsTrustCertsFilePath' cert will allowed to connect to the server, -# though the cert will not be used for client authentication. -tlsAllowInsecureConnection=false - -# Specify the tls protocols the broker will use to negotiate during TLS handshake -# (a comma-separated list of protocol names). -# Examples:- [TLSv1.3, TLSv1.2] -tlsProtocols= - -# Specify the tls cipher the broker will use to negotiate during TLS Handshake -# (a comma-separated list of ciphers). -# Examples:- [TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256] -tlsCiphers= - -# Trusted client certificates are required for to connect TLS -# Reject the Connection if the Client Certificate is not trusted. -# In effect, this requires that all connecting clients perform TLS client -# authentication. -tlsRequireTrustedClientCertOnConnect=false - -### --- KeyStore TLS config variables --- ### -# Enable TLS with KeyStore type configuration in broker. -tlsEnabledWithKeyStore=false - -# TLS Provider for KeyStore type -tlsProvider= - -# TLS KeyStore type configuration in broker: JKS, PKCS12 -tlsKeyStoreType=JKS - -# TLS KeyStore path in broker -tlsKeyStore= - -# TLS KeyStore password for broker -tlsKeyStorePassword= - -# TLS TrustStore type configuration in broker: JKS, PKCS12 -tlsTrustStoreType=JKS - -# TLS TrustStore path in broker -tlsTrustStore= - -# TLS TrustStore password for broker -tlsTrustStorePassword= - -# Whether internal client use KeyStore type to authenticate with Pulsar brokers -brokerClientTlsEnabledWithKeyStore=false - -# The TLS Provider used by internal client to authenticate with other Pulsar brokers -brokerClientSslProvider= - -# TLS TrustStore type configuration for internal client: JKS, PKCS12 -# used by the internal client to authenticate with Pulsar brokers -brokerClientTlsTrustStoreType=JKS - -# TLS TrustStore path for internal client -# used by the internal client to authenticate with Pulsar brokers -brokerClientTlsTrustStore= - -# TLS TrustStore password for internal client, -# used by the internal client to authenticate with Pulsar brokers -brokerClientTlsTrustStorePassword= - -# Specify the tls cipher the internal client will use to negotiate during TLS Handshake -# (a comma-separated list of ciphers) -# e.g. [TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256]. -# used by the internal client to authenticate with Pulsar brokers -brokerClientTlsCiphers= - -# Specify the tls protocols the broker will use to negotiate during TLS handshake -# (a comma-separated list of protocol names). -# e.g. [TLSv1.3, TLSv1.2] -# used by the internal client to authenticate with Pulsar brokers -brokerClientTlsProtocols= - -# Enable or disable system topic -systemTopicEnabled=true - -# Enable or disable topic level policies, topic level policies depends on the system topic -# Please enable the system topic first. -topicLevelPoliciesEnabled=false - -# If a topic remains fenced for this number of seconds, it will be closed forcefully. -# If it is set to 0 or a negative number, the fenced topic will not be closed. -topicFencingTimeoutSeconds=0 - -### --- Authentication --- ### -# Role names that are treated as "proxy roles". If the broker sees a request with -#role as proxyRoles - it will demand to see a valid original principal. -proxyRoles= - -# If this flag is set then the broker authenticates the original Auth data -# else it just accepts the originalPrincipal and authorizes it (if required). -authenticateOriginalAuthData=false - -# Enable authentication -authenticationEnabled=false - -# Authentication provider name list, which is comma separated list of class names -authenticationProviders= - -# Enforce authorization -authorizationEnabled=false - -# Authorization provider fully qualified class-name -authorizationProvider=org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider - -# Allow wildcard matching in authorization -# (wildcard matching only applicable if wildcard-char: -# * presents at first or last position eg: *.pulsar.service, pulsar.service.*) -authorizationAllowWildcardsMatching=false - -# Role names that are treated as "super-user", meaning they will be able to do all admin -# operations and publish/consume from all topics -superUserRoles= - -# Authentication settings of the broker itself. Used when the broker connects to other brokers, -# either in same or other clusters -brokerClientAuthenticationPlugin= -brokerClientAuthenticationParameters= - -# Supported Athenz provider domain names(comma separated) for authentication -athenzDomainNames= - -# When this parameter is not empty, unauthenticated users perform as anonymousUserRole -anonymousUserRole= - - -### --- Token Authentication Provider --- ### - -## Symmetric key -# Configure the secret key to be used to validate auth tokens -# The key can be specified like: -# tokenSecretKey=data:;base64,xxxxxxxxx -# tokenSecretKey=file:///my/secret.key ( Note: key file must be DER-encoded ) -tokenSecretKey= - -## Asymmetric public/private key pair -# Configure the public key to be used to validate auth tokens -# The key can be specified like: -# tokenPublicKey=data:;base64,xxxxxxxxx -# tokenPublicKey=file:///my/public.key ( Note: key file must be DER-encoded ) -tokenPublicKey= - - -# The token "claim" that will be interpreted as the authentication "role" or "principal" by AuthenticationProviderToken (defaults to "sub" if blank) -tokenAuthClaim= - -# The token audience "claim" name, e.g. "aud", that will be used to get the audience from token. -# If not set, audience will not be verified. -tokenAudienceClaim= - -# The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this. -tokenAudience= - -### --- BookKeeper Client --- ### - -# Authentication plugin to use when connecting to bookies -bookkeeperClientAuthenticationPlugin= - -# BookKeeper auth plugin implementation specifics parameters name and values -bookkeeperClientAuthenticationParametersName= -bookkeeperClientAuthenticationParameters= - -# Timeout for BK add / read operations -bookkeeperClientTimeoutInSeconds=30 - -# Number of BookKeeper client worker threads -# Default is Runtime.getRuntime().availableProcessors() -bookkeeperClientNumWorkerThreads= - -# Speculative reads are initiated if a read request doesn't complete within a certain time -# Using a value of 0, is disabling the speculative reads -bookkeeperClientSpeculativeReadTimeoutInMillis=0 - -# Number of channels per bookie -bookkeeperNumberOfChannelsPerBookie=16 - -# Enable bookies health check. Bookies that have more than the configured number of failure within -# the interval will be quarantined for some time. During this period, new ledgers won't be created -# on these bookies -bookkeeperClientHealthCheckEnabled=true -bookkeeperClientHealthCheckIntervalSeconds=60 -bookkeeperClientHealthCheckErrorThresholdPerInterval=5 -bookkeeperClientHealthCheckQuarantineTimeInSeconds=1800 - -#bookie quarantine ratio to avoid all clients quarantine the high pressure bookie servers at the same time -bookkeeperClientQuarantineRatio=1.0 - -# Enable rack-aware bookie selection policy. BK will chose bookies from different racks when -# forming a new bookie ensemble -# This parameter related to ensemblePlacementPolicy in conf/bookkeeper.conf, if enabled, ensemblePlacementPolicy -# should be set to org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy -bookkeeperClientRackawarePolicyEnabled=true - -# Enable region-aware bookie selection policy. BK will chose bookies from -# different regions and racks when forming a new bookie ensemble. -# If enabled, the value of bookkeeperClientRackawarePolicyEnabled is ignored -# This parameter related to ensemblePlacementPolicy in conf/bookkeeper.conf, if enabled, ensemblePlacementPolicy -# should be set to org.apache.bookkeeper.client.RegionAwareEnsemblePlacementPolicy -bookkeeperClientRegionawarePolicyEnabled=false - -# Minimum number of racks per write quorum. BK rack-aware bookie selection policy will try to -# get bookies from at least 'bookkeeperClientMinNumRacksPerWriteQuorum' racks for a write quorum. -bookkeeperClientMinNumRacksPerWriteQuorum=1 - -# Enforces rack-aware bookie selection policy to pick bookies from 'bookkeeperClientMinNumRacksPerWriteQuorum' -# racks for a writeQuorum. -# If BK can't find bookie then it would throw BKNotEnoughBookiesException instead of picking random one. -bookkeeperClientEnforceMinNumRacksPerWriteQuorum=false - -# Enable/disable reordering read sequence on reading entries. -bookkeeperClientReorderReadSequenceEnabled=false - -# Enable bookie isolation by specifying a list of bookie groups to choose from. Any bookie -# outside the specified groups will not be used by the broker -bookkeeperClientIsolationGroups= - -# Enable bookie secondary-isolation group if bookkeeperClientIsolationGroups doesn't -# have enough bookie available. -bookkeeperClientSecondaryIsolationGroups= - -# Minimum bookies that should be available as part of bookkeeperClientIsolationGroups -# else broker will include bookkeeperClientSecondaryIsolationGroups bookies in isolated list. -bookkeeperClientMinAvailableBookiesInIsolationGroups= - -# Set the client security provider factory class name. -# Default: org.apache.bookkeeper.tls.TLSContextFactory -bookkeeperTLSProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory - -# Enable tls authentication with bookie -bookkeeperTLSClientAuthentication=false - -# Supported type: PEM, JKS, PKCS12. Default value: PEM -bookkeeperTLSKeyFileType=PEM - -#Supported type: PEM, JKS, PKCS12. Default value: PEM -bookkeeperTLSTrustCertTypes=PEM - -# Path to file containing keystore password, if the client keystore is password protected. -bookkeeperTLSKeyStorePasswordPath= - -# Path to file containing truststore password, if the client truststore is password protected. -bookkeeperTLSTrustStorePasswordPath= - -# Path for the TLS private key file -bookkeeperTLSKeyFilePath= - -# Path for the TLS certificate file -bookkeeperTLSCertificateFilePath= - -# Path for the trusted TLS certificate file -bookkeeperTLSTrustCertsFilePath= - -# Enable/disable disk weight based placement. Default is false -bookkeeperDiskWeightBasedPlacementEnabled=false - -# Set the interval to check the need for sending an explicit LAC -# A value of '0' disables sending any explicit LACs. Default is 0. -bookkeeperExplicitLacIntervalInMills=0 - -# Use older Bookkeeper wire protocol with bookie -bookkeeperUseV2WireProtocol=true - -# Expose bookkeeper client managed ledger stats to prometheus. default is false -# bookkeeperClientExposeStatsToPrometheus=false - -### --- Managed Ledger --- ### - -# Number of bookies to use when creating a ledger -managedLedgerDefaultEnsembleSize=1 - -# Number of copies to store for each message -managedLedgerDefaultWriteQuorum=1 - -# Number of guaranteed copies (acks to wait before write is complete) -managedLedgerDefaultAckQuorum=1 - -# How frequently to flush the cursor positions that were accumulated due to rate limiting. (seconds). -# Default is 60 seconds -managedLedgerCursorPositionFlushSeconds=60 - -# Default type of checksum to use when writing to BookKeeper. Default is "CRC32C" -# Other possible options are "CRC32", "MAC" or "DUMMY" (no checksum). -managedLedgerDigestType=CRC32C - -# Number of threads to be used for managed ledger tasks dispatching -managedLedgerNumWorkerThreads=4 - -# Number of threads to be used for managed ledger scheduled tasks -managedLedgerNumSchedulerThreads=4 - -# Amount of memory to use for caching data payload in managed ledger. This memory -# is allocated from JVM direct memory and it's shared across all the topics -# running in the same broker. By default, uses 1/5th of available direct memory -managedLedgerCacheSizeMB= - -# Whether we should make a copy of the entry payloads when inserting in cache -managedLedgerCacheCopyEntries=false - -# Threshold to which bring down the cache level when eviction is triggered -managedLedgerCacheEvictionWatermark=0.9 - -# Configure the cache eviction frequency for the managed ledger cache (evictions/sec) -managedLedgerCacheEvictionFrequency=100.0 - -# All entries that have stayed in cache for more than the configured time, will be evicted -managedLedgerCacheEvictionTimeThresholdMillis=1000 - -# Configure the threshold (in number of entries) from where a cursor should be considered 'backlogged' -# and thus should be set as inactive. -managedLedgerCursorBackloggedThreshold=1000 - -# Rate limit the amount of writes generated by consumer acking the messages -managedLedgerDefaultMarkDeleteRateLimit=0.1 - -# Max number of entries to append to a ledger before triggering a rollover -# A ledger rollover is triggered after the min rollover time has passed -# and one of the following conditions is true: -# * The max rollover time has been reached -# * The max entries have been written to the ledger -# * The max ledger size has been written to the ledger -managedLedgerMaxEntriesPerLedger=50000 - -# Minimum time between ledger rollover for a topic -managedLedgerMinLedgerRolloverTimeMinutes=10 - -# Maximum time before forcing a ledger rollover for a topic -managedLedgerMaxLedgerRolloverTimeMinutes=240 - -# Max number of entries to append to a cursor ledger -managedLedgerCursorMaxEntriesPerLedger=50000 - -# Max time before triggering a rollover on a cursor ledger -managedLedgerCursorRolloverTimeInSeconds=14400 - -# Maximum ledger size before triggering a rollover for a topic (MB) -managedLedgerMaxSizePerLedgerMbytes=2048 - -# Max number of "acknowledgment holes" that are going to be persistently stored. -# When acknowledging out of order, a consumer will leave holes that are supposed -# to be quickly filled by acking all the messages. The information of which -# messages are acknowledged is persisted by compressing in "ranges" of messages -# that were acknowledged. After the max number of ranges is reached, the information -# will only be tracked in memory and messages will be redelivered in case of -# crashes. -managedLedgerMaxUnackedRangesToPersist=10000 - -# Max number of "acknowledgment holes" that can be stored in Zookeeper. If number of unack message range is higher -# than this limit then broker will persist unacked ranges into bookkeeper to avoid additional data overhead into -# zookeeper. -managedLedgerMaxUnackedRangesToPersistInZooKeeper=1000 - -# Skip reading non-recoverable/unreadable data-ledger under managed-ledger's list. It helps when data-ledgers gets -# corrupted at bookkeeper and managed-cursor is stuck at that ledger. -autoSkipNonRecoverableData=false - -# operation timeout while updating managed-ledger metadata. -managedLedgerMetadataOperationsTimeoutSeconds=60 - -# Read entries timeout when broker tries to read messages from bookkeeper. -managedLedgerReadEntryTimeoutSeconds=0 - -# Add entry timeout when broker tries to publish message to bookkeeper (0 to disable it). -managedLedgerAddEntryTimeoutSeconds=0 - -# New entries check delay for the cursor under the managed ledger. -# If no new messages in the topic, the cursor will try to check again after the delay time. -# For consumption latency sensitive scenario, can set to a smaller value or set to 0. -# Of course, use a smaller value may degrade consumption throughput. Default is 10ms. -managedLedgerNewEntriesCheckDelayInMillis=10 - -# Use Open Range-Set to cache unacked messages -managedLedgerUnackedRangesOpenCacheSetEnabled=true - -# Managed ledger prometheus stats latency rollover seconds (default: 60s) -managedLedgerPrometheusStatsLatencyRolloverSeconds=60 - -# Whether trace managed ledger task execution time -managedLedgerTraceTaskExecution=true - -# If you want to custom bookie ID or use a dynamic network address for the bookie, -# you can set this option. -# Bookie advertises itself using bookieId rather than -# BookieSocketAddress (hostname:port or IP:port). -# bookieId is a non empty string that can contain ASCII digits and letters ([a-zA-Z9-0]), -# colons, dashes, and dots. -# For more information about bookieId, see http://bookkeeper.apache.org/bps/BP-41-bookieid/. -# bookieId= - -### --- Load balancer --- ### - -loadManagerClassName=org.apache.pulsar.broker.loadbalance.NoopLoadManager - -# Enable load balancer -loadBalancerEnabled=false - -# Percentage of change to trigger load report update -loadBalancerReportUpdateThresholdPercentage=10 - -# maximum interval to update load report -loadBalancerReportUpdateMaxIntervalMinutes=15 - -# Frequency of report to collect -loadBalancerHostUsageCheckIntervalMinutes=1 - -# Load shedding interval. Broker periodically checks whether some traffic should be offload from -# some over-loaded broker to other under-loaded brokers -loadBalancerSheddingIntervalMinutes=1 - -# Prevent the same topics to be shed and moved to other broker more than once within this timeframe -loadBalancerSheddingGracePeriodMinutes=30 - -# Usage threshold to allocate max number of topics to broker -loadBalancerBrokerMaxTopics=50000 - -# Interval to flush dynamic resource quota to ZooKeeper -loadBalancerResourceQuotaUpdateIntervalMinutes=15 - -# enable/disable namespace bundle auto split -loadBalancerAutoBundleSplitEnabled=true - -# enable/disable automatic unloading of split bundles -loadBalancerAutoUnloadSplitBundlesEnabled=true - -# maximum topics in a bundle, otherwise bundle split will be triggered -loadBalancerNamespaceBundleMaxTopics=1000 - -# maximum sessions (producers + consumers) in a bundle, otherwise bundle split will be triggered -loadBalancerNamespaceBundleMaxSessions=1000 - -# maximum msgRate (in + out) in a bundle, otherwise bundle split will be triggered -loadBalancerNamespaceBundleMaxMsgRate=30000 - -# maximum bandwidth (in + out) in a bundle, otherwise bundle split will be triggered -loadBalancerNamespaceBundleMaxBandwidthMbytes=100 - -# maximum number of bundles in a namespace -loadBalancerNamespaceMaximumBundles=128 - -# The broker resource usage threshold. -# When the broker resource usage is greater than the pulsar cluster average resource usage, -# the threshold shedder will be triggered to offload bundles from the broker. -# It only takes effect in the ThresholdShedder strategy. -loadBalancerBrokerThresholdShedderPercentage=10 - -# When calculating new resource usage, the history usage accounts for. -# It only takes effect in the ThresholdShedder strategy. -loadBalancerHistoryResourcePercentage=0.9 - -# The BandWithIn usage weight when calculating new resource usage. -# It only takes effect in the ThresholdShedder strategy. -loadBalancerBandwithInResourceWeight=1.0 - -# The BandWithOut usage weight when calculating new resource usage. -# It only takes effect in the ThresholdShedder strategy. -loadBalancerBandwithOutResourceWeight=1.0 - -# The CPU usage weight when calculating new resource usage. -# It only takes effect in the ThresholdShedder strategy. -loadBalancerCPUResourceWeight=1.0 - -# The heap memory usage weight when calculating new resource usage. -# It only takes effect in the ThresholdShedder strategy. -loadBalancerMemoryResourceWeight=1.0 - -# The direct memory usage weight when calculating new resource usage. -# It only takes effect in the ThresholdShedder strategy. -loadBalancerDirectMemoryResourceWeight=1.0 - -# Bundle unload minimum throughput threshold (MB), avoiding bundle unload frequently. -# It only takes effect in the ThresholdShedder strategy. -loadBalancerBundleUnloadMinThroughputThreshold=10 - -# Time to wait for the unloading of a namespace bundle -namespaceBundleUnloadingTimeoutMs=60000 - -### --- Replication --- ### - -# Enable replication metrics -replicationMetricsEnabled=true - -# Max number of connections to open for each broker in a remote cluster -# More connections host-to-host lead to better throughput over high-latency -# links. -replicationConnectionsPerBroker=16 - -# Replicator producer queue size -replicationProducerQueueSize=1000 - -# Duration to check replication policy to avoid replicator inconsistency -# due to missing ZooKeeper watch (disable with value 0) -replicationPolicyCheckDurationSeconds=600 - -# Default message retention time -defaultRetentionTimeInMinutes=0 - -# Default retention size -defaultRetentionSizeInMB=0 - -# How often to check whether the connections are still alive -keepAliveIntervalSeconds=30 - -### --- WebSocket --- ### - -# Enable the WebSocket API service in broker -webSocketServiceEnabled=true - -# Number of IO threads in Pulsar Client used in WebSocket proxy -webSocketNumIoThreads=8 - -# Number of connections per Broker in Pulsar Client used in WebSocket proxy -webSocketConnectionsPerBroker=8 - -# Time in milliseconds that idle WebSocket session times out -webSocketSessionIdleTimeoutMillis=300000 - -# The maximum size of a text message during parsing in WebSocket proxy -webSocketMaxTextFrameSize=1048576 - -### --- Metrics --- ### - -# Enable topic level metrics -exposeTopicLevelMetricsInPrometheus=true - -# Time in milliseconds that metrics endpoint would time out. Default is 30s. -# Increase it if there are a lot of topics to expose topic-level metrics. -# Set it to 0 to disable timeout. -metricsServletTimeoutMs=30000 - -# Classname of Pluggable JVM GC metrics logger that can log GC specific metrics -# jvmGCMetricsLoggerClassName= - -### --- Broker Web Stats --- ### - -# Enable topic level metrics -exposePublisherStats=true - -# Enable expose the precise backlog stats. -# Set false to use published counter and consumed counter to calculate, this would be more efficient but may be inaccurate. -# Default is false. -exposePreciseBacklogInPrometheus=false - -# Enable splitting topic and partition label in Prometheus. -# If enabled, a topic name will split into 2 parts, one is topic name without partition index, -# another one is partition index, e.g. (topic=xxx, partition=0). -# If the topic is a non-partitioned topic, -1 will be used for the partition index. -# If disabled, one label to represent the topic and partition, e.g. (topic=xxx-partition-0) -# Default is false. - -splitTopicAndPartitionLabelInPrometheus=false - -### --- Deprecated config variables --- ### - -# Deprecated. Use configurationStoreServers -globalZookeeperServers= - -# Deprecated. Use brokerDeleteInactiveTopicsFrequencySeconds -brokerServicePurgeInactiveFrequencyInSeconds=60 - -### --- BookKeeper Configuration --- ##### - -ledgerStorageClass=org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage - -# The maximum netty frame size in bytes. Any message received larger than this will be rejected. The default value is 5MB. -nettyMaxFrameSizeBytes=5253120 - -# Size of Write Cache. Memory is allocated from JVM direct memory. -# Write cache is used to buffer entries before flushing into the entry log -# For good performance, it should be big enough to hold a substantial amount -# of entries in the flush interval -# By default it will be allocated to 1/4th of the available direct memory -dbStorage_writeCacheMaxSizeMb= - -# Size of Read cache. Memory is allocated from JVM direct memory. -# This read cache is pre-filled doing read-ahead whenever a cache miss happens -# By default it will be allocated to 1/4th of the available direct memory -dbStorage_readAheadCacheMaxSizeMb= - -# How many entries to pre-fill in cache after a read cache miss -dbStorage_readAheadCacheBatchSize=1000 - -flushInterval=60000 - -## RocksDB specific configurations -## DbLedgerStorage uses RocksDB to store the indexes from -## (ledgerId, entryId) -> (entryLog, offset) - -# Size of RocksDB block-cache. For best performance, this cache -# should be big enough to hold a significant portion of the index -# database which can reach ~2GB in some cases -# Default is to use 10% of the direct memory size -dbStorage_rocksDB_blockCacheSize= - -# Other RocksDB specific tunables -dbStorage_rocksDB_writeBufferSizeMB=4 -dbStorage_rocksDB_sstSizeInMB=4 -dbStorage_rocksDB_blockSize=4096 -dbStorage_rocksDB_bloomFilterBitsPerKey=10 -dbStorage_rocksDB_numLevels=-1 -dbStorage_rocksDB_numFilesInLevel0=4 -dbStorage_rocksDB_maxSizeInLevel1MB=256 - -# Maximum latency to impose on a journal write to achieve grouping -journalMaxGroupWaitMSec=1 - -# Should the data be fsynced on journal before acknowledgment. -journalSyncData=false - - -# For each ledger dir, maximum disk space which can be used. -# Default is 0.95f. i.e. 95% of disk can be used at most after which nothing will -# be written to that partition. If all ledger dir partions are full, then bookie -# will turn to readonly mode if 'readOnlyModeEnabled=true' is set, else it will -# shutdown. -# Valid values should be in between 0 and 1 (exclusive). -diskUsageThreshold=0.99 - -# The disk free space low water mark threshold. -# Disk is considered full when usage threshold is exceeded. -# Disk returns back to non-full state when usage is below low water mark threshold. -# This prevents it from going back and forth between these states frequently -# when concurrent writes and compaction are happening. This also prevent bookie from -# switching frequently between read-only and read-writes states in the same cases. -diskUsageWarnThreshold=0.99 - -# Whether the bookie allowed to use a loopback interface as its primary -# interface(i.e. the interface it uses to establish its identity)? -# By default, loopback interfaces are not allowed as the primary -# interface. -# Using a loopback interface as the primary interface usually indicates -# a configuration error. For example, its fairly common in some VPS setups -# to not configure a hostname, or to have the hostname resolve to -# 127.0.0.1. If this is the case, then all bookies in the cluster will -# establish their identities as 127.0.0.1:3181, and only one will be able -# to join the cluster. For VPSs configured like this, you should explicitly -# set the listening interface. -allowLoopback=true - -# How long the interval to trigger next garbage collection, in milliseconds -# Since garbage collection is running in background, too frequent gc -# will heart performance. It is better to give a higher number of gc -# interval if there is enough disk capacity. -gcWaitTime=300000 - -# Enable topic auto creation if new producer or consumer connected (disable auto creation with value false) -allowAutoTopicCreation=true - -# The type of topic that is allowed to be automatically created.(partitioned/non-partitioned) -allowAutoTopicCreationType=non-partitioned - -# Enable subscription auto creation if new consumer connected (disable auto creation with value false) -allowAutoSubscriptionCreation=true - -# The number of partitioned topics that is allowed to be automatically created if allowAutoTopicCreationType is partitioned. -defaultNumPartitions=1 - -### --- Transaction config variables --- ### -# Enable transaction coordinator in broker -transactionCoordinatorEnabled=true -transactionMetadataStoreProviderClassName=org.apache.pulsar.transaction.coordinator.impl.MLTransactionMetadataStoreProvider - -# Transaction buffer take snapshot transaction count -transactionBufferSnapshotMaxTransactionCount=1000 - -# Transaction buffer take snapshot interval time -# Unit : millisecond -transactionBufferSnapshotMinTimeInMillis=5000 - -### --- Packages management service configuration variables (begin) --- ### - -# Enable the packages management service or not -enablePackagesManagement=false - -# The packages management service storage service provide -packagesManagementStorageProvider=org.apache.pulsar.packages.management.storage.bookkeeper.BookKeeperPackagesStorageProvider - -# When the packages storage provider is bookkeeper, you can use this configuration to -# control the number of replicas for storing the package -packagesReplicas=1 - -# The bookkeeper ledger root path -packagesManagementLedgerRootPath=/ledgers - -### --- Packages management service configuration variables (end) --- ### diff --git a/flink-connectors/flink-connector-pulsar/src/test/resources/docker/bootstrap.sh b/flink-connectors/flink-connector-pulsar/src/test/resources/docker/bootstrap.sh new file mode 100755 index 0000000000000..fa4e2921a75cf --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/resources/docker/bootstrap.sh @@ -0,0 +1,28 @@ +#!/usr/bin/env bash +# ---------------------------------------------------------------------------- +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# ---------------------------------------------------------------------------- + +# Enable the transaction in standalone config. +sed -i 's/transactionCoordinatorEnabled=false/transactionCoordinatorEnabled=true/g' /pulsar/conf/standalone.conf +sed -i 's/acknowledgmentAtBatchIndexLevelEnabled=false/acknowledgmentAtBatchIndexLevelEnabled=true/g' /pulsar/conf/standalone.conf +sed -i 's/systemTopicEnabled=false/systemTopicEnabled=true/g' /pulsar/conf/standalone.conf +sed -i 's/brokerDeduplicationEnabled=false/brokerDeduplicationEnabled=true/g' /pulsar/conf/standalone.conf + +# Start Pulsar standalone without function worker and streaming storage. +/pulsar/bin/pulsar standalone --no-functions-worker -nss diff --git a/flink-connectors/flink-connector-rabbitmq/pom.xml b/flink-connectors/flink-connector-rabbitmq/pom.xml index 1861c6e87cf90..a0c630934f722 100644 --- a/flink-connectors/flink-connector-rabbitmq/pom.xml +++ b/flink-connectors/flink-connector-rabbitmq/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceITCase.java b/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceITCase.java index cc932d3c1884c..402d659194af8 100644 --- a/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceITCase.java +++ b/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceITCase.java @@ -23,7 +23,6 @@ import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.serialization.SimpleStringSchema; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.core.execution.SavepointFormatType; import org.apache.flink.runtime.execution.ExecutionState; @@ -54,7 +53,6 @@ import org.testcontainers.utility.DockerImageName; import java.io.IOException; -import java.time.Duration; import java.util.List; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; @@ -132,7 +130,6 @@ public void testStopWithSavepoint() throws Exception { info -> info.getExecutionState() == ExecutionState.RUNNING), - Deadline.fromNow(Duration.ofSeconds(10)), 5L); clusterClient @@ -156,10 +153,7 @@ public void testMessageDelivery() throws Exception { source.addSink(CountingSink.getInstance()); final JobGraph jobGraph = env.getStreamGraph().getJobGraph(); JobID jobId = clusterClient.submitJob(jobGraph).get(); - CommonTestUtils.waitUntilCondition( - () -> CountingSink.getCount() == msgs.size(), - Deadline.fromNow(Duration.ofSeconds(30)), - 5L); + CommonTestUtils.waitUntilCondition(() -> CountingSink.getCount() == msgs.size(), 5L); clusterClient.cancel(jobId); } @@ -190,10 +184,7 @@ protected void acknowledgeSessionIDs(List sessionIds) { source.addSink(CountingSink.getInstance()); final JobGraph jobGraph = env.getStreamGraph().getJobGraph(); JobID jobId = clusterClient.submitJob(jobGraph).get(); - CommonTestUtils.waitUntilCondition( - () -> CountingSink.getCount() == msgs.size(), - Deadline.fromNow(Duration.ofSeconds(60)), - 5L); + CommonTestUtils.waitUntilCondition(() -> CountingSink.getCount() == msgs.size(), 5L); clusterClient.cancel(jobId); } diff --git a/flink-connectors/flink-file-sink-common/pom.xml b/flink-connectors/flink-file-sink-common/pom.xml index 081e566ea6a67..e657308a2c963 100644 --- a/flink-connectors/flink-file-sink-common/pom.xml +++ b/flink-connectors/flink-file-sink-common/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-hadoop-compatibility/pom.xml b/flink-connectors/flink-hadoop-compatibility/pom.xml index 54c1b966faf9f..0ec0b56568fdf 100644 --- a/flink-connectors/flink-hadoop-compatibility/pom.xml +++ b/flink-connectors/flink-hadoop-compatibility/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-hcatalog/pom.xml b/flink-connectors/flink-hcatalog/pom.xml index 4351c85ed4a77..93b6b430bed3d 100644 --- a/flink-connectors/flink-hcatalog/pom.xml +++ b/flink-connectors/flink-hcatalog/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-sql-connector-aws-kinesis-firehose/pom.xml b/flink-connectors/flink-sql-connector-aws-kinesis-firehose/pom.xml index 8bcdd1948852a..7dbe7ecfdd520 100644 --- a/flink-connectors/flink-sql-connector-aws-kinesis-firehose/pom.xml +++ b/flink-connectors/flink-sql-connector-aws-kinesis-firehose/pom.xml @@ -23,7 +23,7 @@ flink-connectors org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 @@ -54,6 +54,7 @@ + org.apache.flink:flink-connector-base org.apache.flink:flink-connector-aws-base org.apache.flink:flink-connector-aws-kinesis-firehose software.amazon.awssdk:* diff --git a/flink-connectors/flink-sql-connector-aws-kinesis-streams/pom.xml b/flink-connectors/flink-sql-connector-aws-kinesis-streams/pom.xml index 65fc447aafbc0..13842b151fa17 100644 --- a/flink-connectors/flink-sql-connector-aws-kinesis-streams/pom.xml +++ b/flink-connectors/flink-sql-connector-aws-kinesis-streams/pom.xml @@ -23,7 +23,7 @@ flink-connectors org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 @@ -55,6 +55,7 @@ + org.apache.flink:flink-connector-base org.apache.flink:flink-connector-aws-base org.apache.flink:flink-connector-aws-kinesis-streams software.amazon.awssdk:* diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml index 6aaaf4acba7be..7c17fa1bf3250 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE index de1f4f377a249..f2e61f6fd243d 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE @@ -6,12 +6,12 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) -- com.fasterxml.jackson.core:jackson-core:2.13.0 -- com.fasterxml.jackson.core:jackson-databind:2.13.0 -- com.fasterxml.jackson.core:jackson-annotations:2.13.0 -- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.0 -- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.13.0 -- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.13.0 +- com.fasterxml.jackson.core:jackson-core:2.13.2 +- com.fasterxml.jackson.core:jackson-databind:2.13.2.2 +- com.fasterxml.jackson.core:jackson-annotations:2.13.2 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.2 +- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.13.2 +- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.13.2 - commons-codec:commons-codec:1.15 - commons-logging:commons-logging:1.1.3 - org.apache.httpcomponents:httpasyncclient:4.1.2 diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml index 489e9ea385595..41cb27ba68393 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE index fa04fce164d88..ed85ea436c25a 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -7,12 +7,10 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) - com.carrotsearch:hppc:0.8.1 -- com.fasterxml.jackson.core:jackson-core:2.13.0 -- com.fasterxml.jackson.core:jackson-databind:2.13.0 -- com.fasterxml.jackson.core:jackson-annotations:2.13.0 -- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.0 -- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.13.0 -- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.13.0 +- com.fasterxml.jackson.core:jackson-core:2.13.2 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.2 +- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.13.2 +- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.13.2 - com.github.spullara.mustache.java:compiler:0.9.6 - commons-codec:commons-codec:1.15 - commons-logging:commons-logging:1.1.3 @@ -31,7 +29,6 @@ This project bundles the following dependencies under the Apache Software Licens - org.apache.lucene:lucene-queries:8.7.0 - org.apache.lucene:lucene-queryparser:8.7.0 - org.apache.lucene:lucene-sandbox:8.7.0 -- org.apache.lucene:lucene-spatial:8.7.0 - org.apache.lucene:lucene-spatial-extras:8.7.0 - org.apache.lucene:lucene-spatial3d:8.7.0 - org.apache.lucene:lucene-suggest:8.7.0 @@ -41,7 +38,6 @@ This project bundles the following dependencies under the Apache Software Licens - org.elasticsearch:elasticsearch-geo:7.10.2 - org.elasticsearch:elasticsearch-secure-sm:7.10.2 - org.elasticsearch:elasticsearch-x-content:7.10.2 -- org.elasticsearch:elasticsearch-plugin-classloader:7.10.2 - org.elasticsearch.client:elasticsearch-rest-high-level-client:7.10.2 - org.elasticsearch.client:elasticsearch-rest-client:7.10.2 - org.elasticsearch.plugin:aggs-matrix-stats-client:7.10.2 @@ -49,4 +45,3 @@ This project bundles the following dependencies under the Apache Software Licens - org.elasticsearch.plugin:mapper-extras-client:7.10.2 - org.elasticsearch.plugin:parent-join-client:7.10.2 - org.elasticsearch.plugin:rank-eval-client:7.10.2 -- org.lz4:lz4-java:1.8.0 diff --git a/flink-connectors/flink-sql-connector-hbase-1.4/pom.xml b/flink-connectors/flink-sql-connector-hbase-1.4/pom.xml index 7cf5b8dc882c6..98a05cbe23600 100644 --- a/flink-connectors/flink-sql-connector-hbase-1.4/pom.xml +++ b/flink-connectors/flink-sql-connector-hbase-1.4/pom.xml @@ -23,7 +23,7 @@ under the License. flink-connectors org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 @@ -113,10 +113,6 @@ under the License. - - org.apache.flink.connector.base - org.apache.flink.connector.sql.hbase14.shaded.org.apache.flink.connector.base - org.apache.zookeeper diff --git a/flink-connectors/flink-sql-connector-hbase-2.2/pom.xml b/flink-connectors/flink-sql-connector-hbase-2.2/pom.xml index 9cfae8bd10f6e..addc06aec314a 100644 --- a/flink-connectors/flink-sql-connector-hbase-2.2/pom.xml +++ b/flink-connectors/flink-sql-connector-hbase-2.2/pom.xml @@ -23,7 +23,7 @@ under the License. flink-connectors org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 @@ -117,10 +117,6 @@ under the License. - - org.apache.flink.connector.base - org.apache.flink.connector.sql.hbase22.shaded.org.apache.flink.connector.base - org.apache.zookeeper diff --git a/flink-connectors/flink-sql-connector-hive-1.2.2/pom.xml b/flink-connectors/flink-sql-connector-hive-1.2.2/pom.xml index fa5438ce05acc..23ec6063eb7db 100644 --- a/flink-connectors/flink-sql-connector-hive-1.2.2/pom.xml +++ b/flink-connectors/flink-sql-connector-hive-1.2.2/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-sql-connector-hive-2.2.0/pom.xml b/flink-connectors/flink-sql-connector-hive-2.2.0/pom.xml index cbcbc7a9551bb..c41328f45feb6 100644 --- a/flink-connectors/flink-sql-connector-hive-2.2.0/pom.xml +++ b/flink-connectors/flink-sql-connector-hive-2.2.0/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-sql-connector-hive-2.3.6/pom.xml b/flink-connectors/flink-sql-connector-hive-2.3.6/pom.xml index 9a4d9b6e1a9f0..81dad17c7b14d 100644 --- a/flink-connectors/flink-sql-connector-hive-2.3.6/pom.xml +++ b/flink-connectors/flink-sql-connector-hive-2.3.6/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-sql-connector-hive-3.1.2/pom.xml b/flink-connectors/flink-sql-connector-hive-3.1.2/pom.xml index 907e747d626e2..c65aa9573e060 100644 --- a/flink-connectors/flink-sql-connector-hive-3.1.2/pom.xml +++ b/flink-connectors/flink-sql-connector-hive-3.1.2/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-sql-connector-kafka/pom.xml b/flink-connectors/flink-sql-connector-kafka/pom.xml index 256b9a087ffc1..a331e6c52091d 100644 --- a/flink-connectors/flink-sql-connector-kafka/pom.xml +++ b/flink-connectors/flink-sql-connector-kafka/pom.xml @@ -26,7 +26,7 @@ under the License. flink-connectors org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. @@ -58,6 +58,7 @@ under the License. + org.apache.flink:flink-connector-base org.apache.flink:flink-connector-kafka org.apache.kafka:* diff --git a/flink-connectors/flink-sql-connector-kinesis/pom.xml b/flink-connectors/flink-sql-connector-kinesis/pom.xml index d75a85fe86b30..03c0c439c6951 100644 --- a/flink-connectors/flink-sql-connector-kinesis/pom.xml +++ b/flink-connectors/flink-sql-connector-kinesis/pom.xml @@ -26,7 +26,7 @@ under the License. flink-connectors org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. @@ -58,6 +58,7 @@ under the License. + org.apache.flink:flink-connector-base org.apache.flink:flink-connector-kinesis com.fasterxml.jackson.core:jackson-core com.fasterxml.jackson.core:jackson-databind diff --git a/flink-connectors/flink-sql-connector-kinesis/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-kinesis/src/main/resources/META-INF/NOTICE index fdb21a08d573e..477b69ba68b17 100644 --- a/flink-connectors/flink-sql-connector-kinesis/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-kinesis/src/main/resources/META-INF/NOTICE @@ -14,7 +14,7 @@ This project bundles the following dependencies under the Apache Software Licens - org.apache.commons:commons-lang3:3.3.2 - com.google.guava:guava:29.0-jre - com.google.guava:failureaccess:1.0.1 -- com.fasterxml.jackson.core:jackson-annotations:2.13.0 -- com.fasterxml.jackson.core:jackson-databind:2.13.0 -- com.fasterxml.jackson.core:jackson-core:2.13.0 -- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.0 +- com.fasterxml.jackson.core:jackson-annotations:2.13.2 +- com.fasterxml.jackson.core:jackson-databind:2.13.2.2 +- com.fasterxml.jackson.core:jackson-core:2.13.2 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.2 diff --git a/flink-connectors/flink-sql-connector-pulsar/pom.xml b/flink-connectors/flink-sql-connector-pulsar/pom.xml index 8011f82b4988b..0ec3a32789d8d 100644 --- a/flink-connectors/flink-sql-connector-pulsar/pom.xml +++ b/flink-connectors/flink-sql-connector-pulsar/pom.xml @@ -26,7 +26,7 @@ under the License. flink-connectors org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. @@ -69,6 +69,7 @@ under the License. org.bouncycastle:bcprov-jdk15on org.bouncycastle:bcutil-jdk15on org.slf4j:jul-to-slf4j + com.google.protobuf:* diff --git a/flink-connectors/flink-sql-connector-pulsar/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-pulsar/src/main/resources/META-INF/NOTICE index 56ad187e06b3c..3bcba0e019870 100644 --- a/flink-connectors/flink-sql-connector-pulsar/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-pulsar/src/main/resources/META-INF/NOTICE @@ -6,12 +6,16 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) -- org.apache.pulsar:bouncy-castle-bc:pkg:2.9.1 -- org.apache.pulsar:pulsar-client-admin-api:2.9.1 -- org.apache.pulsar:pulsar-client-all:2.9.1 -- org.apache.pulsar:pulsar-client-api:2.9.1 +- org.apache.pulsar:bouncy-castle-bc:pkg:2.10.0 +- org.apache.pulsar:pulsar-client-admin-api:2.10.0 +- org.apache.pulsar:pulsar-client-all:2.10.0 +- org.apache.pulsar:pulsar-client-api:2.10.0 +- org.slf4j:jul-to-slf4j:1.7.32 + +This project bundles the following dependencies under the Bouncy Castle license. +See bundled license files for details. + - org.bouncycastle:bcpkix-jdk15on:1.69 - org.bouncycastle:bcprov-ext-jdk15on:1.69 - org.bouncycastle:bcprov-jdk15on:1.69 - org.bouncycastle:bcutil-jdk15on:1.69 -- org.slf4j:jul-to-slf4j:1.7.25 diff --git a/flink-connectors/flink-sql-connector-pulsar/src/main/resources/META-INF/licences/LICENSE.bouncycastle b/flink-connectors/flink-sql-connector-pulsar/src/main/resources/META-INF/licences/LICENSE.bouncycastle new file mode 100644 index 0000000000000..e904785dcafae --- /dev/null +++ b/flink-connectors/flink-sql-connector-pulsar/src/main/resources/META-INF/licences/LICENSE.bouncycastle @@ -0,0 +1,7 @@ +Copyright (c) 2000 - 2021 The Legion of the Bouncy Castle Inc. (https://www.bouncycastle.org) + +Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. \ No newline at end of file diff --git a/flink-connectors/flink-sql-connector-rabbitmq/pom.xml b/flink-connectors/flink-sql-connector-rabbitmq/pom.xml index c521fafb323c0..503ad90c06f5f 100644 --- a/flink-connectors/flink-sql-connector-rabbitmq/pom.xml +++ b/flink-connectors/flink-sql-connector-rabbitmq/pom.xml @@ -26,7 +26,7 @@ under the License. flink-connectors org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/pom.xml b/flink-connectors/pom.xml index a1229f38cc1b1..6d9aca55608ee 100644 --- a/flink-connectors/pom.xml +++ b/flink-connectors/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-container/pom.xml b/flink-container/pom.xml index 6c1e1c9e491ea..4f3b534925ff5 100644 --- a/flink-container/pom.xml +++ b/flink-container/pom.xml @@ -24,7 +24,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-contrib/flink-connector-wikiedits/pom.xml b/flink-contrib/flink-connector-wikiedits/pom.xml index efba541cb0506..288497de8297e 100644 --- a/flink-contrib/flink-connector-wikiedits/pom.xml +++ b/flink-contrib/flink-connector-wikiedits/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-contrib - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-contrib/pom.xml b/flink-contrib/pom.xml index 6f8c92557aaaf..ace198c4e1f2d 100644 --- a/flink-contrib/pom.xml +++ b/flink-contrib/pom.xml @@ -27,7 +27,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-core/pom.xml b/flink-core/pom.xml index 8d255adb479a8..78fd16c57d0a0 100644 --- a/flink-core/pom.xml +++ b/flink-core/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java b/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java index 4af9b44233da1..309abfb391917 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java @@ -103,8 +103,13 @@ public enum Type { /** The configuration of state time-to-live(TTL), it is disabled by default. */ @Nonnull private StateTtlConfig ttlConfig = StateTtlConfig.DISABLED; - /** The default value returned by the state when no other value is bound to a key. */ - @Nullable protected transient T defaultValue; + /** + * The default value returned by the state when no other value is bound to a key. + * + * @deprecated To make the semantics more clear, user should manually manage the default value + * if the contents of the state is {@code null} + */ + @Nullable @Deprecated protected transient T defaultValue; // ------------------------------------------------------------------------ @@ -260,6 +265,9 @@ public boolean isQueryable() { *

State user value will expire, become unavailable and be cleaned up in storage depending on * configured {@link StateTtlConfig}. * + *

If enabling the TTL configuration, the field {@link StateDescriptor#defaultValue} will be + * invalid. + * * @param ttlConfig configuration of state TTL */ public void enableTimeToLive(StateTtlConfig ttlConfig) { diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/ExternallyInducedSourceReader.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/ExternallyInducedSourceReader.java index c88bc0f1a157a..f26eb843bc684 100644 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/ExternallyInducedSourceReader.java +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/ExternallyInducedSourceReader.java @@ -24,16 +24,16 @@ import java.util.Optional; /** - * Sources that implement this interface do not trigger checkpoints when receiving a trigger message - * from the checkpoint coordinator, but when their input data/events indicate that a checkpoint + * Sources that implement this interface delay checkpoints when receiving a trigger message from the + * checkpoint coordinator to the point when their input data/events indicate that a checkpoint * should be triggered. * *

The ExternallyInducedSourceReader tells the Flink runtime that a checkpoint needs to be made - * by returning a checkpointId when shouldTriggerCheckpoint() is invoked. + * by returning a checkpointId when {@link #shouldTriggerCheckpoint()} is invoked. * - *

The implementations typically works together with the SplitEnumerator which informs the - * external system to trigger a checkpoint. The external system also needs to forward the Checkpoint - * ID to the source, so the source knows which checkpoint to trigger. + *

The implementations typically works together with the {@link SplitEnumerator} which informs + * the external system to trigger a checkpoint. The external system also needs to forward the + * Checkpoint ID to the source, so the source knows which checkpoint to trigger. * *

Important: It is crucial that all parallel source tasks trigger their checkpoints at * roughly the same time. Otherwise this leads to performance issues due to long checkpoint diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java index 30f48803fdaa0..28218a35dfc6c 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java @@ -1040,7 +1040,7 @@ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundE } cl = Thread.currentThread().getContextClassLoader(); - subclassSerializerCache = new HashMap, TypeSerializer>(); + subclassSerializerCache = new HashMap<>(); } // -------------------------------------------------------------------------------------------- diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeSerializerFactory.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeSerializerFactory.java index d9a84f65e50f1..9b8f541daa32d 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeSerializerFactory.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeSerializerFactory.java @@ -36,8 +36,6 @@ public final class RuntimeSerializerFactory private TypeSerializer serializer; - private boolean firstSerializer = true; - private Class clazz; // Because we read the class from the TaskConfig and instantiate ourselves @@ -62,7 +60,6 @@ public void writeParametersToConfig(Configuration config) { } } - @SuppressWarnings("unchecked") @Override public void readParametersFromConfig(Configuration config, ClassLoader cl) throws ClassNotFoundException { @@ -71,12 +68,8 @@ public void readParametersFromConfig(Configuration config, ClassLoader cl) } try { - this.clazz = - (Class) InstantiationUtil.readObjectFromConfig(config, CONFIG_KEY_CLASS, cl); - this.serializer = - (TypeSerializer) - InstantiationUtil.readObjectFromConfig(config, CONFIG_KEY_SER, cl); - firstSerializer = true; + this.clazz = InstantiationUtil.readObjectFromConfig(config, CONFIG_KEY_CLASS, cl); + this.serializer = InstantiationUtil.readObjectFromConfig(config, CONFIG_KEY_SER, cl); } catch (ClassNotFoundException e) { throw e; } catch (Exception e) { @@ -87,12 +80,7 @@ public void readParametersFromConfig(Configuration config, ClassLoader cl) @Override public TypeSerializer getSerializer() { if (this.serializer != null) { - if (firstSerializer) { - firstSerializer = false; - return this.serializer; - } else { - return this.serializer.duplicate(); - } + return this.serializer.duplicate(); } else { throw new RuntimeException( "SerializerFactory has not been initialized from configuration."); diff --git a/flink-core/src/main/java/org/apache/flink/configuration/CleanupOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/CleanupOptions.java index 49d5dacfb28df..1efc4c59d1724 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/CleanupOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/CleanupOptions.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.annotation.docs.ConfigGroup; import org.apache.flink.annotation.docs.ConfigGroups; +import org.apache.flink.annotation.docs.Documentation; import org.apache.flink.configuration.description.Description; import org.apache.flink.configuration.description.TextElement; @@ -93,7 +94,9 @@ public static String extractAlphaNumericCharacters(String paramName) { Collectors.joining( ", ")) + ": Cleanup is only performed once. No retry " - + "will be initiated in case of failure.", + + "will be initiated in case of failure. The job " + + "artifacts (and the job's JobResultStore entry) have " + + "to be cleaned up manually in case of a failure.", NONE_PARAM_VALUES.stream() .map(TextElement::code) .collect(Collectors.toList()) @@ -105,7 +108,9 @@ public static String extractAlphaNumericCharacters(String paramName) { "%s, %s: Cleanup attempts will be separated by a fixed " + "interval up to the point where the cleanup is " + "considered successful or a set amount of retries " - + "is reached.", + + "is reached. Reaching the configured limit means that " + + "the job artifacts (and the job's JobResultStore entry) " + + "might need to be cleaned up manually.", code(FIXED_DELAY_LABEL), code( extractAlphaNumericCharacters( @@ -115,7 +120,9 @@ public static String extractAlphaNumericCharacters(String paramName) { + "triggers the cleanup with an exponentially " + "increasing delay up to the point where the " + "cleanup succeeded or a set amount of retries " - + "is reached.", + + "is reached. Reaching the configured limit means that " + + "the job artifacts (and the job's JobResultStore entry) " + + "might need to be cleaned up manually.", code(EXPONENTIAL_DELAY_LABEL), code( extractAlphaNumericCharacters( @@ -125,22 +132,26 @@ public static String extractAlphaNumericCharacters(String paramName) { + "retry strategy with the given default values.") .build()); + @Documentation.OverrideDefault("infinite") public static final ConfigOption CLEANUP_STRATEGY_FIXED_DELAY_ATTEMPTS = ConfigOptions.key(createFixedDelayParameterPrefix("attempts")) .intType() - .defaultValue(1) + .defaultValue(Integer.MAX_VALUE) .withDescription( Description.builder() .text( "The number of times that Flink retries the cleanup " - + "before giving up if %s has been set to %s.", + + "before giving up if %s has been set to %s. " + + "Reaching the configured limit means that " + + "the job artifacts (and the job's JobResultStore entry) " + + "might need to be cleaned up manually.", code(CLEANUP_STRATEGY_PARAM), code(FIXED_DELAY_LABEL)) .build()); public static final ConfigOption CLEANUP_STRATEGY_FIXED_DELAY_DELAY = ConfigOptions.key(createFixedDelayParameterPrefix("delay")) .durationType() - .defaultValue(Duration.ofSeconds(1)) + .defaultValue(Duration.ofMinutes(1)) .withDescription( Description.builder() .text( @@ -180,16 +191,19 @@ public static String extractAlphaNumericCharacters(String paramName) { code(EXPONENTIAL_DELAY_LABEL)) .build()); + @Documentation.OverrideDefault("infinite") public static final ConfigOption CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_ATTEMPTS = ConfigOptions.key(createExponentialBackoffParameterPrefix("attempts")) .intType() - .noDefaultValue() + .defaultValue(Integer.MAX_VALUE) .withDescription( Description.builder() .text( "The number of times a failed cleanup is retried " - + "if %s has been set to %s. (no value means: " - + "infinitely).", + + "if %s has been set to %s. Reaching the " + + "configured limit means that the job artifacts " + + "(and the job's JobResultStore entry) " + + "might need to be cleaned up manually.", code(CLEANUP_STRATEGY_PARAM), code(EXPONENTIAL_DELAY_LABEL)) .build()); diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java index c14399ed271bb..e7c441ccc5194 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java @@ -1732,6 +1732,11 @@ public final class ConfigConstants { /** The environment variable name which contains the location of the opt directory. */ public static final String ENV_FLINK_OPT_DIR = "FLINK_OPT_DIR"; + /** + * The default Flink opt directory if none has been specified via {@link #ENV_FLINK_OPT_DIR}. + */ + public static final String DEFAULT_FLINK_OPT_DIR = "opt"; + /** The environment variable name which contains the location of the plugins folder. */ public static final String ENV_FLINK_PLUGINS_DIR = "FLINK_PLUGINS_DIR"; diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigUtils.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigUtils.java index 1aa36d135a27e..1c275bc02e5ae 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigUtils.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigUtils.java @@ -23,11 +23,14 @@ import javax.annotation.Nullable; +import java.lang.reflect.Field; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.HashSet; import java.util.List; import java.util.Objects; +import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; @@ -132,5 +135,25 @@ public static List decodeListFromConfig( return result; } + public static Set> getAllConfigOptions(Class configOptionsClass) + throws IllegalStateException { + final Set> options = new HashSet<>(); + final Field[] fields = configOptionsClass.getDeclaredFields(); + for (Field field : fields) { + if (field.getType() == ConfigOption.class) { + try { + options.add((ConfigOption) field.get(configOptionsClass)); + } catch (IllegalAccessException e) { + throw new IllegalStateException( + "The config option definition for field " + + field.getName() + + " is not accessible.", + e); + } + } + } + return options; + } + private ConfigUtils() {} } diff --git a/flink-core/src/main/java/org/apache/flink/configuration/StateChangelogOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/StateChangelogOptions.java index 588e6707b7422..ef2fc62794da0 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/StateChangelogOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/StateChangelogOptions.java @@ -34,7 +34,8 @@ public class StateChangelogOptions { .defaultValue(Duration.ofMinutes(10)) .withDescription( "Defines the interval in milliseconds to perform " - + "periodic materialization for state backend."); + + "periodic materialization for state backend. " + + "The periodic materialization will be disabled when the value is negative"); @Documentation.Section(Documentation.Sections.STATE_BACKEND_CHANGELOG) public static final ConfigOption MATERIALIZATION_MAX_FAILURES_ALLOWED = diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java index 1503b4bc2c9c8..a4da43416fdfc 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java @@ -21,9 +21,12 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.core.fs.FSDataOutputStream; +import javax.annotation.Nonnull; + import java.io.File; import java.io.FileOutputStream; import java.io.IOException; +import java.nio.channels.ClosedChannelException; /** * The LocalDataOutputStream class is a wrapper class for a data output stream to the @@ -35,6 +38,8 @@ public class LocalDataOutputStream extends FSDataOutputStream { /** The file output stream used to write data. */ private final FileOutputStream fos; + private boolean closed = false; + /** * Constructs a new LocalDataOutputStream object from a given {@link File} object. * @@ -47,31 +52,49 @@ public LocalDataOutputStream(final File file) throws IOException { @Override public void write(final int b) throws IOException { + checkOpen(); + fos.write(b); + } + + @Override + public void write(@Nonnull final byte[] b) throws IOException { + checkOpen(); fos.write(b); } @Override public void write(final byte[] b, final int off, final int len) throws IOException { + checkOpen(); fos.write(b, off, len); } @Override public void close() throws IOException { + closed = true; fos.close(); } @Override public void flush() throws IOException { + checkOpen(); fos.flush(); } @Override public void sync() throws IOException { + checkOpen(); fos.getFD().sync(); } @Override public long getPos() throws IOException { + checkOpen(); return fos.getChannel().position(); } + + private void checkOpen() throws IOException { + if (closed) { + throw new ClosedChannelException(); + } + } } diff --git a/flink-core/src/test/java/org/apache/flink/core/fs/local/LocalFileSystemTest.java b/flink-core/src/test/java/org/apache/flink/core/fs/local/LocalFileSystemTest.java index 2a7bcc6c5d539..51bc888440b55 100644 --- a/flink-core/src/test/java/org/apache/flink/core/fs/local/LocalFileSystemTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/fs/local/LocalFileSystemTest.java @@ -28,6 +28,7 @@ import org.apache.flink.util.ExecutorUtils; import org.apache.flink.util.FileUtils; import org.apache.flink.util.TestLogger; +import org.apache.flink.util.function.ThrowingConsumer; import org.apache.commons.lang3.RandomStringUtils; import org.junit.Assume; @@ -39,6 +40,7 @@ import java.io.FileInputStream; import java.io.FileOutputStream; import java.io.IOException; +import java.nio.channels.ClosedChannelException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -389,6 +391,44 @@ public void testCreatingFileInCurrentDirectoryWithRelativePath() throws IOExcept } } + @Test(expected = ClosedChannelException.class) + public void testFlushMethodFailsOnClosedOutputStream() throws IOException { + testMethodCallFailureOnClosedStream(FSDataOutputStream::flush); + } + + @Test(expected = ClosedChannelException.class) + public void testWriteIntegerMethodFailsOnClosedOutputStream() throws IOException { + testMethodCallFailureOnClosedStream(os -> os.write(0)); + } + + @Test(expected = ClosedChannelException.class) + public void testWriteBytesMethodFailsOnClosedOutputStream() throws IOException { + testMethodCallFailureOnClosedStream(os -> os.write(new byte[0])); + } + + @Test(expected = ClosedChannelException.class) + public void testWriteBytesSubArrayMethodFailsOnClosedOutputStream() throws IOException { + testMethodCallFailureOnClosedStream(os -> os.write(new byte[0], 0, 0)); + } + + @Test(expected = ClosedChannelException.class) + public void testGetPosMethodFailsOnClosedOutputStream() throws IOException { + testMethodCallFailureOnClosedStream(FSDataOutputStream::getPos); + } + + private void testMethodCallFailureOnClosedStream( + ThrowingConsumer callback) throws IOException { + final FileSystem fs = FileSystem.getLocalFileSystem(); + final FSDataOutputStream outputStream = + fs.create( + new Path( + temporaryFolder.getRoot().toString(), + "close_fs_test_" + UUID.randomUUID()), + WriteMode.OVERWRITE); + outputStream.close(); + callback.accept(outputStream); + } + private Collection createTargetDirectories( File root, int directoryDepth, int numberDirectories) { final StringBuilder stringBuilder = new StringBuilder(); diff --git a/flink-dist-scala/pom.xml b/flink-dist-scala/pom.xml index b571e1f94aba9..b86f2b68983af 100644 --- a/flink-dist-scala/pom.xml +++ b/flink-dist-scala/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml index 22bbb0967d5cd..57e02b6a7683a 100644 --- a/flink-dist/pom.xml +++ b/flink-dist/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-dist/src/main/resources/flink-conf.yaml b/flink-dist/src/main/resources/flink-conf.yaml index e1ce1286c35bc..a8c0b7bb1a712 100644 --- a/flink-dist/src/main/resources/flink-conf.yaml +++ b/flink-dist/src/main/resources/flink-conf.yaml @@ -38,6 +38,8 @@ jobmanager.rpc.port: 6123 # The host interface the JobManager will bind to. My default, this is localhost, and will prevent # the JobManager from communicating outside the machine/container it is running on. +# On YARN this setting will be ignored if it is set to 'localhost', defaulting to 0.0.0.0. +# On Kubernetes this setting will be ignored, defaulting to 0.0.0.0. # # To enable this, set the bind-host address to one that has access to an outside facing network # interface, such as 0.0.0.0. @@ -53,6 +55,8 @@ jobmanager.memory.process.size: 1600m # The host interface the TaskManager will bind to. By default, this is localhost, and will prevent # the TaskManager from communicating outside the machine/container it is running on. +# On YARN this setting will be ignored if it is set to 'localhost', defaulting to 0.0.0.0. +# On Kubernetes this setting will be ignored, defaulting to 0.0.0.0. # # To enable this, set the bind-host address to one that has access to an outside facing network # interface, such as 0.0.0.0. diff --git a/flink-docs/pom.xml b/flink-docs/pom.xml index c9450f3560c8b..b16043140710e 100644 --- a/flink-docs/pom.xml +++ b/flink-docs/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java b/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java index 4602cb60af5ee..d86959e5d9232 100644 --- a/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java +++ b/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java @@ -104,6 +104,15 @@ public class ConfigOptionsDocGenerator { new OptionsClassLocation( "flink-connectors/flink-connector-pulsar", "org.apache.flink.connector.pulsar.source"), + new OptionsClassLocation( + "flink-connectors/flink-connector-pulsar", + "org.apache.flink.connector.pulsar.sink"), + new OptionsClassLocation( + "flink-connectors/flink-connector-pulsar", + "org.apache.flink.connector.pulsar.table.catalog"), + new OptionsClassLocation( + "flink-connectors/flink-connector-pulsar", + "org.apache.flink.connector.pulsar.table"), new OptionsClassLocation( "flink-libraries/flink-cep", "org.apache.flink.cep.configuration"), new OptionsClassLocation( diff --git a/flink-dstl/flink-dstl-dfs/pom.xml b/flink-dstl/flink-dstl-dfs/pom.xml index 6f059e1a0263b..083d6ceb90b1a 100644 --- a/flink-dstl/flink-dstl-dfs/pom.xml +++ b/flink-dstl/flink-dstl-dfs/pom.xml @@ -27,7 +27,7 @@ under the License. org.apache.flink flink-dstl - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogStorage.java b/flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogStorage.java index 9c8bcdc60b1eb..adeabed918a20 100644 --- a/flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogStorage.java +++ b/flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogStorage.java @@ -19,6 +19,7 @@ import org.apache.flink.annotation.Experimental; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.operators.MailboxExecutor; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.io.AvailabilityProvider; @@ -91,11 +92,12 @@ public FsStateChangelogStorage( } @Override - public FsStateChangelogWriter createWriter(String operatorID, KeyGroupRange keyGroupRange) { + public FsStateChangelogWriter createWriter( + String operatorID, KeyGroupRange keyGroupRange, MailboxExecutor mailboxExecutor) { UUID logId = new UUID(0, logIdGenerator.getAndIncrement()); LOG.info("createWriter for operator {}/{}: {}", operatorID, keyGroupRange, logId); return new FsStateChangelogWriter( - logId, keyGroupRange, uploader, preEmptivePersistThresholdInBytes); + logId, keyGroupRange, uploader, preEmptivePersistThresholdInBytes, mailboxExecutor); } @Override diff --git a/flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogWriter.java b/flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogWriter.java index b6537a7b3de72..87e6240fd23e6 100644 --- a/flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogWriter.java +++ b/flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogWriter.java @@ -18,6 +18,7 @@ package org.apache.flink.changelog.fs; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.operators.MailboxExecutor; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.changelog.fs.StateChangeUploadScheduler.UploadTask; import org.apache.flink.runtime.state.KeyGroupRange; @@ -32,7 +33,6 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nullable; -import javax.annotation.concurrent.GuardedBy; import javax.annotation.concurrent.NotThreadSafe; import java.io.IOException; @@ -46,7 +46,6 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; -import static java.util.concurrent.CompletableFuture.completedFuture; import static org.apache.flink.util.IOUtils.closeAllQuietly; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -94,12 +93,7 @@ class FsStateChangelogWriter implements StateChangelogWriter uploadCompletionListeners = new ArrayList<>(); /** Current {@link SequenceNumber}. */ @@ -109,7 +103,6 @@ class FsStateChangelogWriter implements StateChangelogWriter notUploaded = new TreeMap<>(); /** Uploaded changes, ready for use in snapshots. */ - @GuardedBy("lock") private final NavigableMap uploaded = new TreeMap<>(); /** * Highest {@link SequenceNumber} for which upload has failed (won't be restarted), inclusive. */ - @Nullable - @GuardedBy("lock") - private Tuple2 highestFailed; + @Nullable private Tuple2 highestFailed; - @GuardedBy("lock") private boolean closed; + private final MailboxExecutor mailboxExecutor; + FsStateChangelogWriter( UUID logId, KeyGroupRange keyGroupRange, StateChangeUploadScheduler uploader, - long preEmptivePersistThresholdInBytes) { + long preEmptivePersistThresholdInBytes, + MailboxExecutor mailboxExecutor) { this.logId = logId; this.keyGroupRange = keyGroupRange; this.uploader = uploader; this.preEmptivePersistThresholdInBytes = preEmptivePersistThresholdInBytes; + this.mailboxExecutor = mailboxExecutor; } @Override @@ -194,87 +187,90 @@ public CompletableFuture persist(SequenceNumber private CompletableFuture persistInternal(SequenceNumber from) throws IOException { - synchronized (lock) { - ensureCanPersist(from); - rollover(); - Map toUpload = drainTailMap(notUploaded, from); - NavigableMap readyToReturn = uploaded.tailMap(from, true); - LOG.debug("collected readyToReturn: {}, toUpload: {}", readyToReturn, toUpload); - - SequenceNumberRange range = SequenceNumberRange.generic(from, activeSequenceNumber); - if (range.size() == readyToReturn.size()) { - checkState(toUpload.isEmpty()); - return completedFuture(buildHandle(keyGroupRange, readyToReturn, 0L)); - } else { - CompletableFuture future = - new CompletableFuture<>(); - uploadCompletionListeners.add( - new UploadCompletionListener(keyGroupRange, range, readyToReturn, future)); - if (!toUpload.isEmpty()) { - uploader.upload( - new UploadTask( - toUpload.values(), - this::handleUploadSuccess, - this::handleUploadFailure)); - } - return future; + ensureCanPersist(from); + rollover(); + Map toUpload = drainTailMap(notUploaded, from); + NavigableMap readyToReturn = uploaded.tailMap(from, true); + LOG.debug("collected readyToReturn: {}, toUpload: {}", readyToReturn, toUpload); + + SequenceNumberRange range = SequenceNumberRange.generic(from, activeSequenceNumber); + if (range.size() == readyToReturn.size()) { + checkState(toUpload.isEmpty()); + return CompletableFuture.completedFuture(buildHandle(keyGroupRange, readyToReturn, 0L)); + } else { + CompletableFuture future = new CompletableFuture<>(); + uploadCompletionListeners.add( + new UploadCompletionListener(keyGroupRange, range, readyToReturn, future)); + if (!toUpload.isEmpty()) { + UploadTask uploadTask = + new UploadTask( + toUpload.values(), + this::handleUploadSuccess, + this::handleUploadFailure); + uploader.upload(uploadTask); } + return future; } } private void handleUploadFailure(List failedSqn, Throwable throwable) { - synchronized (lock) { - if (closed) { - return; - } - uploadCompletionListeners.removeIf( - listener -> listener.onFailure(failedSqn, throwable)); - failedSqn.stream() - .max(Comparator.naturalOrder()) - .filter(sqn -> sqn.compareTo(lowestSequenceNumber) >= 0) - .filter(sqn -> highestFailed == null || sqn.compareTo(highestFailed.f0) > 0) - .ifPresent(sqn -> highestFailed = Tuple2.of(sqn, throwable)); - } + mailboxExecutor.execute( + () -> { + if (closed) { + return; + } + uploadCompletionListeners.removeIf( + listener -> listener.onFailure(failedSqn, throwable)); + failedSqn.stream() + .max(Comparator.naturalOrder()) + .filter(sqn -> sqn.compareTo(lowestSequenceNumber) >= 0) + .filter( + sqn -> + highestFailed == null + || sqn.compareTo(highestFailed.f0) > 0) + .ifPresent(sqn -> highestFailed = Tuple2.of(sqn, throwable)); + }, + "handleUploadFailure"); } private void handleUploadSuccess(List results) { - synchronized (lock) { - if (closed) { - results.forEach( - r -> closeAllQuietly(() -> r.getStreamStateHandle().discardState())); - } else { - uploadCompletionListeners.removeIf(listener -> listener.onSuccess(results)); - for (UploadResult result : results) { - if (result.sequenceNumber.compareTo(lowestSequenceNumber) >= 0) { - uploaded.put(result.sequenceNumber, result); + mailboxExecutor.execute( + () -> { + if (closed) { + results.forEach( + r -> + closeAllQuietly( + () -> r.getStreamStateHandle().discardState())); + } else { + uploadCompletionListeners.removeIf(listener -> listener.onSuccess(results)); + for (UploadResult result : results) { + if (result.sequenceNumber.compareTo(lowestSequenceNumber) >= 0) { + uploaded.put(result.sequenceNumber, result); + } + } } - } - } - } + }, + "handleUploadSuccess"); } @Override public void close() { LOG.debug("close {}", logId); - synchronized (lock) { - checkState(!closed); - closed = true; - activeChangeSet.clear(); - activeChangeSetSize = 0; - notUploaded.clear(); - uploaded.clear(); - } + checkState(!closed); + closed = true; + activeChangeSet.clear(); + activeChangeSetSize = 0; + notUploaded.clear(); + uploaded.clear(); } @Override public void truncate(SequenceNumber to) { LOG.debug("truncate {} to sqn {} (excl.)", logId, to); checkArgument(to.compareTo(activeSequenceNumber) <= 0); - synchronized (lock) { - lowestSequenceNumber = to; - notUploaded.headMap(lowestSequenceNumber, false).clear(); - uploaded.headMap(lowestSequenceNumber, false).clear(); - } + lowestSequenceNumber = to; + notUploaded.headMap(lowestSequenceNumber, false).clear(); + uploaded.headMap(lowestSequenceNumber, false).clear(); } private void rollover() { diff --git a/flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeFormat.java b/flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeFormat.java index ba04f595bded0..ff2149301e42c 100644 --- a/flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeFormat.java +++ b/flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeFormat.java @@ -91,7 +91,7 @@ public CloseableIterator read(StreamStateHandle handle, long offset throws IOException { FSDataInputStream stream = handle.openInputStream(); DataInputViewStreamWrapper input = wrap(stream); - if (stream.getPos() != offset) { + if (offset != 0) { LOG.debug("seek from {} to {}", stream.getPos(), offset); input.skipBytesToRead((int) offset); } diff --git a/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/BatchingStateChangeUploadSchedulerTest.java b/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/BatchingStateChangeUploadSchedulerTest.java index e12f1f2af116b..f0d9cf2d70c32 100644 --- a/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/BatchingStateChangeUploadSchedulerTest.java +++ b/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/BatchingStateChangeUploadSchedulerTest.java @@ -220,19 +220,19 @@ public void testRetryOnTimeout() throws Exception { new ManuallyTriggeredScheduledExecutorService(); BlockingUploader uploader = new BlockingUploader(); try (BatchingStateChangeUploadScheduler store = - scheduler(numAttempts, executorService, uploader, 10)) { + scheduler(numAttempts, executorService, uploader, 50)) { store.upload(upload); Deadline deadline = Deadline.fromNow(Duration.ofMinutes(5)); while (uploader.getUploadsCount() < numAttempts - 1 && deadline.hasTimeLeft()) { executorService.triggerScheduledTasks(); executorService.triggerAll(); - Thread.sleep(10); + Thread.sleep(1); // should be less than timeout to avoid all attempts timing out } uploader.unblock(); while (!upload.finished.get() && deadline.hasTimeLeft()) { executorService.triggerScheduledTasks(); executorService.triggerAll(); - Thread.sleep(10); + Thread.sleep(1); } } @@ -429,7 +429,7 @@ private Tuple2> uploadAsync(int limit, TestScena return Tuple2.of(thread, future); } - private static final class BlockingUploader implements StateChangeUploader { + static final class BlockingUploader implements StateChangeUploader { private final AtomicBoolean blocking = new AtomicBoolean(true); private final AtomicInteger uploadsCounter = new AtomicInteger(); @@ -449,7 +449,7 @@ public UploadTasksResult upload(Collection tasks) { @Override public void close() {} - private void unblock() { + void unblock() { blocking.set(false); } diff --git a/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/ChangelogStorageMetricsTest.java b/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/ChangelogStorageMetricsTest.java index a4e9e551537ef..9c031175f74ee 100644 --- a/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/ChangelogStorageMetricsTest.java +++ b/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/ChangelogStorageMetricsTest.java @@ -23,6 +23,7 @@ import org.apache.flink.core.testutils.ManuallyTriggeredScheduledExecutorService; import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.HistogramStatistics; +import org.apache.flink.runtime.mailbox.SyncMailboxExecutor; import org.apache.flink.runtime.metrics.groups.TaskManagerJobMetricGroup; import org.apache.flink.runtime.metrics.util.TestingMetricRegistry; import org.apache.flink.runtime.state.changelog.SequenceNumber; @@ -61,7 +62,7 @@ public void testUploadsCounter() throws Exception { try (FsStateChangelogStorage storage = new FsStateChangelogStorage( Path.fromLocalFile(temporaryFolder.newFolder()), false, 100, metrics)) { - FsStateChangelogWriter writer = storage.createWriter("writer", EMPTY_KEY_GROUP_RANGE); + FsStateChangelogWriter writer = createWriter(storage); int numUploads = 5; for (int i = 0; i < numUploads; i++) { @@ -82,7 +83,7 @@ public void testUploadSizes() throws Exception { try (FsStateChangelogStorage storage = new FsStateChangelogStorage( Path.fromLocalFile(temporaryFolder.newFolder()), false, 100, metrics)) { - FsStateChangelogWriter writer = storage.createWriter("writer", EMPTY_KEY_GROUP_RANGE); + FsStateChangelogWriter writer = createWriter(storage); // upload single byte to infer header size SequenceNumber from = writer.nextSequenceNumber(); @@ -108,7 +109,7 @@ public void testUploadFailuresCounter() throws Exception { new ChangelogStorageMetricGroup(createUnregisteredTaskManagerJobMetricGroup()); try (FsStateChangelogStorage storage = new FsStateChangelogStorage(Path.fromLocalFile(file), false, 100, metrics)) { - FsStateChangelogWriter writer = storage.createWriter("writer", EMPTY_KEY_GROUP_RANGE); + FsStateChangelogWriter writer = createWriter(storage); int numUploads = 5; for (int i = 0; i < numUploads; i++) { @@ -149,7 +150,9 @@ public void testUploadBatchSizes() throws Exception { FsStateChangelogStorage storage = new FsStateChangelogStorage(batcher, Integer.MAX_VALUE); FsStateChangelogWriter[] writers = new FsStateChangelogWriter[numWriters]; for (int i = 0; i < numWriters; i++) { - writers[i] = storage.createWriter(Integer.toString(i), EMPTY_KEY_GROUP_RANGE); + writers[i] = + storage.createWriter( + Integer.toString(i), EMPTY_KEY_GROUP_RANGE, new SyncMailboxExecutor()); } try { @@ -190,7 +193,7 @@ public void testAttemptsPerUpload() throws Exception { metrics); FsStateChangelogStorage storage = new FsStateChangelogStorage(batcher, Integer.MAX_VALUE); - FsStateChangelogWriter writer = storage.createWriter("writer", EMPTY_KEY_GROUP_RANGE); + FsStateChangelogWriter writer = createWriter(storage); try { for (int upload = 0; upload < numUploads; upload++) { @@ -198,12 +201,12 @@ public void testAttemptsPerUpload() throws Exception { writer.append(0, new byte[] {0, 1, 2, 3}); writer.persist(from).get(); } - HistogramStatistics histogram = metrics.getAttemptsPerUpload().getStatistics(); - assertEquals(maxAttempts, histogram.getMin()); - assertEquals(maxAttempts, histogram.getMax()); } finally { storage.close(); } + HistogramStatistics histogram = metrics.getAttemptsPerUpload().getStatistics(); + assertEquals(maxAttempts, histogram.getMin()); + assertEquals(maxAttempts, histogram.getMax()); } @Test @@ -242,7 +245,7 @@ public void testQueueSize() throws Exception { metrics); try (FsStateChangelogStorage storage = new FsStateChangelogStorage(batcher, Long.MAX_VALUE)) { - FsStateChangelogWriter writer = storage.createWriter("writer", EMPTY_KEY_GROUP_RANGE); + FsStateChangelogWriter writer = createWriter(storage); int numUploads = 11; for (int i = 0; i < numUploads; i++) { SequenceNumber from = writer.nextSequenceNumber(); @@ -288,4 +291,8 @@ public void close() { attemptsPerTask.clear(); } } + + private FsStateChangelogWriter createWriter(FsStateChangelogStorage storage) { + return storage.createWriter("writer", EMPTY_KEY_GROUP_RANGE, new SyncMailboxExecutor()); + } } diff --git a/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/FsStateChangelogStorageTest.java b/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/FsStateChangelogStorageTest.java index 6179be65af05c..602155dc38a90 100644 --- a/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/FsStateChangelogStorageTest.java +++ b/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/FsStateChangelogStorageTest.java @@ -17,14 +17,24 @@ package org.apache.flink.changelog.fs; +import org.apache.flink.changelog.fs.BatchingStateChangeUploadSchedulerTest.BlockingUploader; import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.changelog.StateChangelogStorage; +import org.apache.flink.runtime.state.changelog.StateChangelogWriter; import org.apache.flink.runtime.state.changelog.inmemory.StateChangelogStorageTest; +import org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor; +import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxExecutorImpl; +import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailboxImpl; +import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; import static org.apache.flink.changelog.fs.UnregisteredChangelogStorageMetricGroup.createUnregisteredChangelogStorageMetricGroup; @@ -46,4 +56,62 @@ protected StateChangelogStorage getFactory() throws IOException { 1024 * 1024 * 10, createUnregisteredChangelogStorageMetricGroup()); } + + /** + * Provoke a deadlock between task and uploader threads which might happen during waiting for + * capacity and upload completion. + */ + @Test + public void testDeadlockOnUploadCompletion() throws Throwable { + int capacity = 10; // in bytes, allow the first two uploads without waiting (see below) + CountDownLatch remainingUploads = new CountDownLatch(3); + BlockingUploader blockingUploader = new BlockingUploader(); + CompletableFuture unblockFuture = new CompletableFuture<>(); + new Thread( + () -> { + try { + remainingUploads.await(); + blockingUploader.unblock(); + unblockFuture.complete(null); + } catch (Throwable e) { + unblockFuture.completeExceptionally(e); + } + }) + .start(); + MailboxExecutorImpl mailboxExecutor = + new MailboxExecutorImpl( + new TaskMailboxImpl(), 0, StreamTaskActionExecutor.IMMEDIATE); + try (BatchingStateChangeUploadScheduler scheduler = + new BatchingStateChangeUploadScheduler( + 0, // schedule immediately + 0, // schedule immediately + RetryPolicy.NONE, + blockingUploader, + 1, + capacity, + createUnregisteredChangelogStorageMetricGroup()) { + @Override + public void upload(UploadTask uploadTask) throws IOException { + remainingUploads.countDown(); + super.upload(uploadTask); + } + }; + StateChangelogWriter writer = + new FsStateChangelogStorage(scheduler, 0 /* persist immediately */) + .createWriter( + new OperatorID().toString(), + KeyGroupRange.of(0, 0), + mailboxExecutor); ) { + // 1. start with 1-byte request - releasing only it will NOT allow proceeding in 3, but + // still involves completion callback, which can deadlock + writer.append(0, new byte[1]); + // 2. exceed capacity + writer.append(0, new byte[capacity]); + // 3. current thread will block until both previous requests are completed + // verify that completion can proceed while this thread is waiting + writer.append(0, new byte[1]); + } + // check unblocking thread exit status + unblockFuture.join(); + } } diff --git a/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/FsStateChangelogWriterSqnTest.java b/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/FsStateChangelogWriterSqnTest.java index 4d4a37d2ccd7b..f3e576dcc97ee 100644 --- a/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/FsStateChangelogWriterSqnTest.java +++ b/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/FsStateChangelogWriterSqnTest.java @@ -17,6 +17,7 @@ package org.apache.flink.changelog.fs; +import org.apache.flink.runtime.mailbox.SyncMailboxExecutor; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.changelog.SequenceNumber; import org.apache.flink.runtime.state.changelog.StateChangelogWriter; @@ -86,7 +87,8 @@ public void runTest() throws IOException { KeyGroupRange.of(0, 0), StateChangeUploadScheduler.directScheduler( new TestingStateChangeUploader()), - Long.MAX_VALUE)) { + Long.MAX_VALUE, + new SyncMailboxExecutor())) { if (test.withAppend) { append(writer); } diff --git a/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/FsStateChangelogWriterTest.java b/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/FsStateChangelogWriterTest.java index 64addf79ff776..b0a49dd5cabae 100644 --- a/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/FsStateChangelogWriterTest.java +++ b/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/FsStateChangelogWriterTest.java @@ -17,6 +17,7 @@ package org.apache.flink.changelog.fs; +import org.apache.flink.runtime.mailbox.SyncMailboxExecutor; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.changelog.ChangelogStateHandleStreamImpl; import org.apache.flink.runtime.state.changelog.SequenceNumber; @@ -211,7 +212,8 @@ private void withWriter( UUID.randomUUID(), KeyGroupRange.of(KEY_GROUP, KEY_GROUP), StateChangeUploadScheduler.directScheduler(uploader), - appendPersistThreshold)) { + appendPersistThreshold, + new SyncMailboxExecutor())) { test.accept(writer, uploader); } } diff --git a/flink-dstl/pom.xml b/flink-dstl/pom.xml index dd0ff26fbd21d..52fbb16382bc3 100644 --- a/flink-dstl/pom.xml +++ b/flink-dstl/pom.xml @@ -27,7 +27,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-batch-sql-test/pom.xml b/flink-end-to-end-tests/flink-batch-sql-test/pom.xml index f38bd2ac8b852..5ed81b3d0c5ff 100644 --- a/flink-end-to-end-tests/flink-batch-sql-test/pom.xml +++ b/flink-end-to-end-tests/flink-batch-sql-test/pom.xml @@ -24,7 +24,7 @@ flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-cli-test/pom.xml b/flink-end-to-end-tests/flink-cli-test/pom.xml index 6d046e66e3c6a..33cfe71ffe362 100644 --- a/flink-end-to-end-tests/flink-cli-test/pom.xml +++ b/flink-end-to-end-tests/flink-cli-test/pom.xml @@ -24,7 +24,7 @@ flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-confluent-schema-registry/pom.xml b/flink-end-to-end-tests/flink-confluent-schema-registry/pom.xml index 55fbbf2bb45e4..eed32bfb5a361 100644 --- a/flink-end-to-end-tests/flink-confluent-schema-registry/pom.xml +++ b/flink-end-to-end-tests/flink-confluent-schema-registry/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/pom.xml b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/pom.xml index 3368a307cf241..6c1303956615e 100644 --- a/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/pom.xml +++ b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-end-to-end-tests - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-dataset-allround-test/pom.xml b/flink-end-to-end-tests/flink-dataset-allround-test/pom.xml index 44f5e7372d7e7..e47e3361e3c05 100644 --- a/flink-end-to-end-tests/flink-dataset-allround-test/pom.xml +++ b/flink-end-to-end-tests/flink-dataset-allround-test/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-end-to-end-tests - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-dataset-fine-grained-recovery-test/pom.xml b/flink-end-to-end-tests/flink-dataset-fine-grained-recovery-test/pom.xml index 1e33f3084e471..5e1aa4a2671f7 100644 --- a/flink-end-to-end-tests/flink-dataset-fine-grained-recovery-test/pom.xml +++ b/flink-end-to-end-tests/flink-dataset-fine-grained-recovery-test/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-end-to-end-tests - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/pom.xml b/flink-end-to-end-tests/flink-datastream-allround-test/pom.xml index 4b400806be58d..537b9b9437d62 100644 --- a/flink-end-to-end-tests/flink-datastream-allround-test/pom.xml +++ b/flink-end-to-end-tests/flink-datastream-allround-test/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-end-to-end-tests - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-distributed-cache-via-blob-test/pom.xml b/flink-end-to-end-tests/flink-distributed-cache-via-blob-test/pom.xml index 21bdec7ca363b..57a370fe4457f 100644 --- a/flink-end-to-end-tests/flink-distributed-cache-via-blob-test/pom.xml +++ b/flink-end-to-end-tests/flink-distributed-cache-via-blob-test/pom.xml @@ -24,7 +24,7 @@ flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-elasticsearch6-test/pom.xml b/flink-end-to-end-tests/flink-elasticsearch6-test/pom.xml index ee1aecaf70a37..4d42aea40ef8a 100644 --- a/flink-end-to-end-tests/flink-elasticsearch6-test/pom.xml +++ b/flink-end-to-end-tests/flink-elasticsearch6-test/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-end-to-end-tests - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-elasticsearch7-test/pom.xml b/flink-end-to-end-tests/flink-elasticsearch7-test/pom.xml index f210e3b0b7374..f7d09566e7e99 100644 --- a/flink-end-to-end-tests/flink-elasticsearch7-test/pom.xml +++ b/flink-end-to-end-tests/flink-elasticsearch7-test/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-end-to-end-tests - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-firehose/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-firehose/pom.xml index f32d6f1e097e6..d1a54d53ccbe1 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-firehose/pom.xml +++ b/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-firehose/pom.xml @@ -23,7 +23,7 @@ flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 @@ -51,14 +51,6 @@ test test-jar - - - org.apache.flink - flink-connector-aws-base - ${project.version} - test - - org.apache.flink flink-table-planner_${scala.binary.version} @@ -80,19 +72,19 @@ com.fasterxml.jackson.core jackson-core - 2.13.0 + 2.13.2 com.fasterxml.jackson.core jackson-annotations - 2.13.0 + 2.13.2 com.fasterxml.jackson.core jackson-databind - 2.13.0 + 2.13.2.2 diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-streams/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-streams/pom.xml index d2aa1323fd142..a7b1d9594f0cc 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-streams/pom.xml +++ b/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-streams/pom.xml @@ -23,7 +23,7 @@ flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 @@ -65,13 +65,6 @@ jackson-databind test - - - org.apache.flink - flink-connector-aws-base - ${project.version} - test - diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/pom.xml index fce2debda9029..6296e64556529 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/pom.xml +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/pom.xml @@ -23,7 +23,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml index b2e6ee57e9d1a..90dc526ec60ec 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkContainerTestEnvironment.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkContainerTestEnvironment.java index 1fbc8ae131b66..9b78dec617687 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkContainerTestEnvironment.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkContainerTestEnvironment.java @@ -18,7 +18,6 @@ package org.apache.flink.tests.util.flink; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.testframe.environment.ClusterControllable; import org.apache.flink.connector.testframe.environment.TestEnvironment; @@ -35,7 +34,6 @@ import org.slf4j.LoggerFactory; import java.net.URL; -import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -149,8 +147,7 @@ public void triggerTaskManagerFailover(JobClient jobClient, Runnable afterFailAc flinkContainers .getRestClusterClient() .getJobDetails(jobClient.getJobID()) - .get(), - Deadline.fromNow(Duration.ofMinutes(5))); + .get()); afterFailAction.run(); }); } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-hbase/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-hbase/pom.xml index 6b76dddfffbd5..5bc2de72b4cf6 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-hbase/pom.xml +++ b/flink-end-to-end-tests/flink-end-to-end-tests-hbase/pom.xml @@ -23,7 +23,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/pom.xml index 424b106f4d3e9..8b8d794cebc2b 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/pom.xml +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/pom.xml @@ -23,7 +23,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 @@ -31,7 +31,7 @@ under the License. Flink : E2E Tests : Pulsar - 2.9.1 + 2.10.0 1.69 @@ -53,6 +53,17 @@ under the License. jaxb-api ${jaxb.api.version} + + org.apache.flink + flink-connector-test-utils + ${project.version} + + + com.google.guava + guava + + + org.apache.flink flink-connector-pulsar @@ -70,9 +81,6 @@ under the License. org.apache.maven.plugins maven-surefire-plugin - - true - org.apache.maven.plugins @@ -176,6 +184,14 @@ under the License. jar ${project.build.directory}/dependencies + + org.apache.flink + flink-connector-test-utils + ${project.version} + flink-connector-testing.jar + jar + ${project.build.directory}/dependencies + diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSinkE2ECase.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSinkE2ECase.java new file mode 100644 index 0000000000000..a19f5931b587a --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSinkE2ECase.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.pulsar; + +import org.apache.flink.connector.pulsar.testutils.PulsarTestContextFactory; +import org.apache.flink.connector.pulsar.testutils.sink.PulsarSinkTestContext; +import org.apache.flink.connector.pulsar.testutils.sink.PulsarSinkTestSuiteBase; +import org.apache.flink.connector.testframe.junit.annotations.TestContext; +import org.apache.flink.connector.testframe.junit.annotations.TestEnv; +import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem; +import org.apache.flink.connector.testframe.junit.annotations.TestSemantics; +import org.apache.flink.streaming.api.CheckpointingMode; +import org.apache.flink.tests.util.pulsar.common.FlinkContainerWithPulsarEnvironment; +import org.apache.flink.tests.util.pulsar.common.PulsarContainerTestEnvironment; +import org.apache.flink.testutils.junit.FailsOnJava11; + +import org.junit.experimental.categories.Category; + +/** Pulsar sink E2E test based on connector testing framework. */ +@SuppressWarnings("unused") +@Category(value = {FailsOnJava11.class}) +public class PulsarSinkE2ECase extends PulsarSinkTestSuiteBase { + + @TestSemantics + CheckpointingMode[] semantics = + new CheckpointingMode[] { + CheckpointingMode.EXACTLY_ONCE, CheckpointingMode.AT_LEAST_ONCE + }; + + // Defines TestEnvironment + @TestEnv + FlinkContainerWithPulsarEnvironment flink = new FlinkContainerWithPulsarEnvironment(1, 6); + + // Defines ConnectorExternalSystem. + @TestExternalSystem + PulsarContainerTestEnvironment pulsar = new PulsarContainerTestEnvironment(flink); + + @TestContext + PulsarTestContextFactory sinkContext = + new PulsarTestContextFactory<>(pulsar, PulsarSinkTestContext::new); +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceOrderedE2ECase.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceOrderedE2ECase.java index 234c1a01cd623..ea6a982044b8f 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceOrderedE2ECase.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceOrderedE2ECase.java @@ -25,10 +25,10 @@ import org.apache.flink.connector.testframe.junit.annotations.TestSemantics; import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase; import org.apache.flink.streaming.api.CheckpointingMode; -import org.apache.flink.tests.util.pulsar.cases.ExclusiveSubscriptionContext; -import org.apache.flink.tests.util.pulsar.cases.FailoverSubscriptionContext; import org.apache.flink.tests.util.pulsar.common.FlinkContainerWithPulsarEnvironment; import org.apache.flink.tests.util.pulsar.common.PulsarContainerTestEnvironment; +import org.apache.flink.tests.util.pulsar.source.ExclusiveSubscriptionContext; +import org.apache.flink.tests.util.pulsar.source.FailoverSubscriptionContext; import org.apache.flink.testutils.junit.FailsOnJava11; import org.junit.experimental.categories.Category; @@ -37,6 +37,7 @@ * Pulsar E2E test based on connector testing framework. It's used for Failover & Exclusive * subscription. */ +@SuppressWarnings("unused") @Category(value = {FailsOnJava11.class}) public class PulsarSourceOrderedE2ECase extends SourceTestSuiteBase { diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceUnorderedE2ECase.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceUnorderedE2ECase.java index 50390486dd69c..6bf4fc0c43c7a 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceUnorderedE2ECase.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceUnorderedE2ECase.java @@ -24,16 +24,21 @@ import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem; import org.apache.flink.connector.testframe.junit.annotations.TestSemantics; import org.apache.flink.streaming.api.CheckpointingMode; -import org.apache.flink.tests.util.pulsar.cases.KeySharedSubscriptionContext; -import org.apache.flink.tests.util.pulsar.cases.SharedSubscriptionContext; import org.apache.flink.tests.util.pulsar.common.FlinkContainerWithPulsarEnvironment; import org.apache.flink.tests.util.pulsar.common.PulsarContainerTestEnvironment; import org.apache.flink.tests.util.pulsar.common.UnorderedSourceTestSuiteBase; +import org.apache.flink.tests.util.pulsar.source.KeySharedSubscriptionContext; +import org.apache.flink.tests.util.pulsar.source.SharedSubscriptionContext; +import org.apache.flink.testutils.junit.FailsOnJava11; + +import org.junit.experimental.categories.Category; /** * Pulsar E2E test based on connector testing framework. It's used for Shared & Key_Shared * subscription. */ +@SuppressWarnings("unused") +@Category(value = {FailsOnJava11.class}) public class PulsarSourceUnorderedE2ECase extends UnorderedSourceTestSuiteBase { // Defines the Semantic. @@ -49,12 +54,10 @@ public class PulsarSourceUnorderedE2ECase extends UnorderedSourceTestSuiteBase shared = new PulsarTestContextFactory<>(pulsar, SharedSubscriptionContext::new); - @SuppressWarnings("unused") @TestContext PulsarTestContextFactory keyShared = new PulsarTestContextFactory<>(pulsar, KeySharedSubscriptionContext::new); diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/KeySharedSubscriptionContext.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/KeySharedSubscriptionContext.java deleted file mode 100644 index 5ad369bcf03f3..0000000000000 --- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/KeySharedSubscriptionContext.java +++ /dev/null @@ -1,144 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.tests.util.pulsar.cases; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.api.connector.source.Source; -import org.apache.flink.connector.pulsar.source.PulsarSource; -import org.apache.flink.connector.pulsar.source.PulsarSourceBuilder; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange; -import org.apache.flink.connector.pulsar.source.enumerator.topic.range.FixedRangeGenerator; -import org.apache.flink.connector.pulsar.testutils.PulsarTestContext; -import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; -import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter; -import org.apache.flink.connector.testframe.external.source.TestingSourceSettings; -import org.apache.flink.tests.util.pulsar.common.KeyedPulsarPartitionDataWriter; - -import org.apache.pulsar.client.api.RegexSubscriptionMode; -import org.apache.pulsar.client.api.SubscriptionType; -import org.apache.pulsar.common.util.Murmur3_32Hash; - -import java.net.URL; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -import static java.util.Collections.singletonList; -import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; -import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.RANGE_SIZE; -import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.pulsarSchema; -import static org.apache.pulsar.client.api.Schema.STRING; - -/** We would consume from test splits by using {@link SubscriptionType#Key_Shared} subscription. */ -public class KeySharedSubscriptionContext extends PulsarTestContext { - - private int index = 0; - - private final List writers = new ArrayList<>(); - - // Message keys. - private final String key1; - private final String key2; - - public KeySharedSubscriptionContext(PulsarTestEnvironment environment) { - this(environment, Collections.emptyList()); - } - - public KeySharedSubscriptionContext( - PulsarTestEnvironment environment, List connectorJarPaths) { - super(environment, connectorJarPaths); - - // Init message keys. - this.key1 = randomAlphabetic(8); - String newKey2; - do { - newKey2 = randomAlphabetic(8); - } while (keyHash(key1) == keyHash(newKey2)); - this.key2 = newKey2; - } - - @Override - protected String displayName() { - return "consuming message by Key_Shared"; - } - - @Override - public Source createSource(TestingSourceSettings sourceSettings) { - int keyHash = keyHash(key1); - TopicRange range = new TopicRange(keyHash, keyHash); - - PulsarSourceBuilder builder = - PulsarSource.builder() - .setDeserializationSchema(pulsarSchema(STRING)) - .setServiceUrl(operator.serviceUrl()) - .setAdminUrl(operator.adminUrl()) - .setTopicPattern( - "pulsar-[0-9]+-key-shared", RegexSubscriptionMode.AllTopics) - .setSubscriptionType(SubscriptionType.Key_Shared) - .setSubscriptionName("pulsar-key-shared") - .setRangeGenerator(new FixedRangeGenerator(singletonList(range))); - if (sourceSettings.getBoundedness() == Boundedness.BOUNDED) { - // Using latest stop cursor for making sure the source could be stopped. - builder.setBoundedStopCursor(StopCursor.latest()); - } - - return builder.build(); - } - - @Override - public ExternalSystemSplitDataWriter createSourceSplitDataWriter( - TestingSourceSettings sourceSettings) { - String topicName = "pulsar-" + index + "-key-shared"; - operator.createTopic(topicName, 1); - index++; - - String partitionName = TopicNameUtils.topicNameWithPartition(topicName, 0); - KeyedPulsarPartitionDataWriter writer = - new KeyedPulsarPartitionDataWriter(operator, partitionName, key1, key2); - writers.add(writer); - - return writer; - } - - @Override - public List generateTestData( - TestingSourceSettings sourceSettings, int splitIndex, long seed) { - return generateStringTestData(splitIndex, seed); - } - - @Override - public TypeInformation getProducedType() { - return TypeInformation.of(String.class); - } - - @Override - public void close() { - for (KeyedPulsarPartitionDataWriter writer : writers) { - writer.close(); - } - writers.clear(); - } - - private int keyHash(String key) { - return Murmur3_32Hash.getInstance().makeHash(key.getBytes()) % RANGE_SIZE; - } -} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/SharedSubscriptionContext.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/SharedSubscriptionContext.java deleted file mode 100644 index 1a2db6694d1fb..0000000000000 --- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/SharedSubscriptionContext.java +++ /dev/null @@ -1,116 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.tests.util.pulsar.cases; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.api.connector.source.Source; -import org.apache.flink.connector.pulsar.source.PulsarSource; -import org.apache.flink.connector.pulsar.source.PulsarSourceBuilder; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; -import org.apache.flink.connector.pulsar.testutils.PulsarPartitionDataWriter; -import org.apache.flink.connector.pulsar.testutils.PulsarTestContext; -import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; -import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter; -import org.apache.flink.connector.testframe.external.source.TestingSourceSettings; - -import org.apache.pulsar.client.api.RegexSubscriptionMode; -import org.apache.pulsar.client.api.SubscriptionType; - -import java.net.URL; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.pulsarSchema; -import static org.apache.pulsar.client.api.Schema.STRING; - -/** We would consuming from test splits by using {@link SubscriptionType#Shared} subscription. */ -public class SharedSubscriptionContext extends PulsarTestContext { - - private int index = 0; - - private final List writers = new ArrayList<>(); - - public SharedSubscriptionContext(PulsarTestEnvironment environment) { - this(environment, Collections.emptyList()); - } - - public SharedSubscriptionContext( - PulsarTestEnvironment environment, List connectorJarPaths) { - super(environment, connectorJarPaths); - } - - @Override - protected String displayName() { - return "consuming message by Shared"; - } - - @Override - public Source createSource(TestingSourceSettings sourceSettings) { - PulsarSourceBuilder builder = - PulsarSource.builder() - .setDeserializationSchema(pulsarSchema(STRING)) - .setServiceUrl(operator.serviceUrl()) - .setAdminUrl(operator.adminUrl()) - .setTopicPattern("pulsar-[0-9]+-shared", RegexSubscriptionMode.AllTopics) - .setSubscriptionType(SubscriptionType.Shared) - .setSubscriptionName("pulsar-shared"); - if (sourceSettings.getBoundedness() == Boundedness.BOUNDED) { - // Using latest stop cursor for making sure the source could be stopped. - builder.setBoundedStopCursor(StopCursor.latest()); - } - - return builder.build(); - } - - @Override - public ExternalSystemSplitDataWriter createSourceSplitDataWriter( - TestingSourceSettings sourceSettings) { - String topicName = "pulsar-" + index + "-shared"; - operator.createTopic(topicName, 1); - index++; - - String partitionName = TopicNameUtils.topicNameWithPartition(topicName, 0); - PulsarPartitionDataWriter writer = new PulsarPartitionDataWriter(operator, partitionName); - writers.add(writer); - - return writer; - } - - @Override - public List generateTestData( - TestingSourceSettings sourceSettings, int splitIndex, long seed) { - return generateStringTestData(splitIndex, seed); - } - - @Override - public TypeInformation getProducedType() { - return TypeInformation.of(String.class); - } - - @Override - public void close() { - for (PulsarPartitionDataWriter writer : writers) { - writer.close(); - } - writers.clear(); - } -} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/FlinkContainerWithPulsarEnvironment.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/FlinkContainerWithPulsarEnvironment.java index 9a1c6dd6f0d6d..14d8cbdd12cec 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/FlinkContainerWithPulsarEnvironment.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/FlinkContainerWithPulsarEnvironment.java @@ -43,7 +43,8 @@ public FlinkContainerWithPulsarEnvironment(int numTaskManagers, int numSlotsPerT resourcePath("bcutil-jdk15on.jar"), resourcePath("bcprov-ext-jdk15on.jar"), resourcePath("jaxb-api.jar"), - resourcePath("jul-to-slf4j.jar")); + resourcePath("jul-to-slf4j.jar"), + resourcePath("flink-connector-testing.jar")); } private static String resourcePath(String jarName) { diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/KeyedPulsarPartitionDataWriter.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/KeyedPulsarPartitionDataWriter.java index e431e4c89d0e1..bdcf8160a968d 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/KeyedPulsarPartitionDataWriter.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/KeyedPulsarPartitionDataWriter.java @@ -35,23 +35,26 @@ public class KeyedPulsarPartitionDataWriter implements ExternalSystemSplitDataWr private final PulsarRuntimeOperator operator; private final String fullTopicName; - private final String key1; - private final String key2; + private final String keyToRead; + private final String keyToExclude; public KeyedPulsarPartitionDataWriter( - PulsarRuntimeOperator operator, String fullTopicName, String key1, String key2) { + PulsarRuntimeOperator operator, + String fullTopicName, + String keyToRead, + String keyToExclude) { this.operator = operator; this.fullTopicName = fullTopicName; - this.key1 = key1; - this.key2 = key2; + this.keyToRead = keyToRead; + this.keyToExclude = keyToExclude; } @Override public void writeRecords(List records) { - operator.sendMessages(fullTopicName, Schema.STRING, key1, records); + List newRecords = records.stream().map(a -> a + keyToRead).collect(toList()); + operator.sendMessages(fullTopicName, Schema.STRING, keyToExclude, newRecords); - List newRecords = records.stream().map(a -> a + key1).collect(toList()); - operator.sendMessages(fullTopicName, Schema.STRING, key2, newRecords); + operator.sendMessages(fullTopicName, Schema.STRING, keyToRead, records); } @Override diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/UnorderedSourceTestSuiteBase.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/UnorderedSourceTestSuiteBase.java index 01527ea54824d..59b05f0955c54 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/UnorderedSourceTestSuiteBase.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/UnorderedSourceTestSuiteBase.java @@ -18,69 +18,67 @@ package org.apache.flink.tests.util.pulsar.common; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.api.connector.source.Source; +import org.apache.flink.connector.testframe.environment.ClusterControllable; import org.apache.flink.connector.testframe.environment.TestEnvironment; -import org.apache.flink.connector.testframe.environment.TestEnvironmentSettings; -import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter; import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext; -import org.apache.flink.connector.testframe.external.source.TestingSourceSettings; -import org.apache.flink.connector.testframe.junit.extensions.ConnectorTestingExtension; -import org.apache.flink.connector.testframe.junit.extensions.TestCaseInvocationContextProvider; +import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase; import org.apache.flink.streaming.api.CheckpointingMode; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.util.TestLoggerExtension; -import org.junit.jupiter.api.DisplayName; -import org.junit.jupiter.api.TestInstance; -import org.junit.jupiter.api.TestTemplate; -import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.Disabled; -import java.util.List; -import java.util.concurrent.ThreadLocalRandom; +/** A source test template for testing the messages which could be consumed in an unordered way. */ +public abstract class UnorderedSourceTestSuiteBase extends SourceTestSuiteBase { -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.containsInAnyOrder; + private static final String DISABLE_REASON = + "UnorderedSourceTestSuiteBase don't support any test in SourceTestSuiteBase."; -/** A source test template for testing the messages which could be consumed in a unordered way. */ -@ExtendWith({ - ConnectorTestingExtension.class, - TestLoggerExtension.class, - TestCaseInvocationContextProvider.class -}) -@TestInstance(TestInstance.Lifecycle.PER_CLASS) -public abstract class UnorderedSourceTestSuiteBase { + @Override + @Disabled(DISABLE_REASON) + public void testMultipleSplits( + TestEnvironment testEnv, + DataStreamSourceExternalContext externalContext, + CheckpointingMode semantic) {} - @TestTemplate - @DisplayName("Test source with one split and four consumers") - public void testOneSplitWithMultipleConsumers( - TestEnvironment testEnv, DataStreamSourceExternalContext externalContext) - throws Exception { - TestingSourceSettings sourceSettings = - TestingSourceSettings.builder() - .setBoundedness(Boundedness.BOUNDED) - .setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE) - .build(); - TestEnvironmentSettings envOptions = - TestEnvironmentSettings.builder() - .setConnectorJarPaths(externalContext.getConnectorJarPaths()) - .build(); - List testData = - externalContext.generateTestData( - sourceSettings, 0, ThreadLocalRandom.current().nextLong()); - ExternalSystemSplitDataWriter writer = - externalContext.createSourceSplitDataWriter(sourceSettings); - writer.writeRecords(testData); + @Override + @Disabled(DISABLE_REASON) + public void testSavepoint( + TestEnvironment testEnv, + DataStreamSourceExternalContext externalContext, + CheckpointingMode semantic) {} - Source source = externalContext.createSource(sourceSettings); - StreamExecutionEnvironment execEnv = testEnv.createExecutionEnvironment(envOptions); - List results = - execEnv.fromSource(source, WatermarkStrategy.noWatermarks(), "Pulsar source") - .setParallelism(4) - .executeAndCollect( - "Source single split with four readers.", testData.size()); + @Override + @Disabled(DISABLE_REASON) + public void testScaleUp( + TestEnvironment testEnv, + DataStreamSourceExternalContext externalContext, + CheckpointingMode semantic) {} - assertThat(results, containsInAnyOrder(testData.toArray())); - } + @Override + @Disabled(DISABLE_REASON) + public void testScaleDown( + TestEnvironment testEnv, + DataStreamSourceExternalContext externalContext, + CheckpointingMode semantic) {} + + @Override + @Disabled(DISABLE_REASON) + public void testSourceMetrics( + TestEnvironment testEnv, + DataStreamSourceExternalContext externalContext, + CheckpointingMode semantic) {} + + @Override + @Disabled(DISABLE_REASON) + public void testIdleReader( + TestEnvironment testEnv, + DataStreamSourceExternalContext externalContext, + CheckpointingMode semantic) {} + + @Override + @Disabled(DISABLE_REASON) + public void testTaskManagerFailure( + TestEnvironment testEnv, + DataStreamSourceExternalContext externalContext, + ClusterControllable controller, + CheckpointingMode semantic) {} } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/ExclusiveSubscriptionContext.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/ExclusiveSubscriptionContext.java similarity index 71% rename from flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/ExclusiveSubscriptionContext.java rename to flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/ExclusiveSubscriptionContext.java index 6fea0c9c9e52d..4906ad6cc7e03 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/ExclusiveSubscriptionContext.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/ExclusiveSubscriptionContext.java @@ -16,32 +16,23 @@ * limitations under the License. */ -package org.apache.flink.tests.util.pulsar.cases; +package org.apache.flink.tests.util.pulsar.source; import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; -import org.apache.flink.connector.pulsar.testutils.cases.MultipleTopicTemplateContext; +import org.apache.flink.connector.pulsar.testutils.source.cases.MultipleTopicConsumingContext; import org.apache.pulsar.client.api.SubscriptionType; -import java.net.URL; -import java.util.Collections; -import java.util.List; - /** We would consume from test splits by using {@link SubscriptionType#Exclusive} subscription. */ -public class ExclusiveSubscriptionContext extends MultipleTopicTemplateContext { +public class ExclusiveSubscriptionContext extends MultipleTopicConsumingContext { public ExclusiveSubscriptionContext(PulsarTestEnvironment environment) { - this(environment, Collections.emptyList()); - } - - public ExclusiveSubscriptionContext( - PulsarTestEnvironment environment, List connectorJarPaths) { - super(environment, connectorJarPaths); + super(environment); } @Override protected String displayName() { - return "consuming message by Exclusive"; + return "consume message by Exclusive"; } @Override diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/FailoverSubscriptionContext.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/FailoverSubscriptionContext.java similarity index 71% rename from flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/FailoverSubscriptionContext.java rename to flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/FailoverSubscriptionContext.java index c47348861c11d..3134db4bdda57 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/FailoverSubscriptionContext.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/FailoverSubscriptionContext.java @@ -16,32 +16,23 @@ * limitations under the License. */ -package org.apache.flink.tests.util.pulsar.cases; +package org.apache.flink.tests.util.pulsar.source; import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; -import org.apache.flink.connector.pulsar.testutils.cases.MultipleTopicTemplateContext; +import org.apache.flink.connector.pulsar.testutils.source.cases.MultipleTopicConsumingContext; import org.apache.pulsar.client.api.SubscriptionType; -import java.net.URL; -import java.util.Collections; -import java.util.List; - /** We would consume from test splits by using {@link SubscriptionType#Failover} subscription. */ -public class FailoverSubscriptionContext extends MultipleTopicTemplateContext { +public class FailoverSubscriptionContext extends MultipleTopicConsumingContext { public FailoverSubscriptionContext(PulsarTestEnvironment environment) { - this(environment, Collections.emptyList()); - } - - public FailoverSubscriptionContext( - PulsarTestEnvironment environment, List connectorJarPaths) { - super(environment, connectorJarPaths); + super(environment); } @Override protected String displayName() { - return "consuming message by Failover"; + return "consume message by Failover"; } @Override diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/KeySharedSubscriptionContext.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/KeySharedSubscriptionContext.java new file mode 100644 index 0000000000000..6d579b5eccfd2 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/KeySharedSubscriptionContext.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.pulsar.source; + +import org.apache.flink.connector.pulsar.source.PulsarSourceBuilder; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange; +import org.apache.flink.connector.pulsar.source.enumerator.topic.range.FixedRangeGenerator; +import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; +import org.apache.flink.connector.pulsar.testutils.source.cases.MultipleTopicConsumingContext; +import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter; +import org.apache.flink.connector.testframe.external.source.TestingSourceSettings; +import org.apache.flink.tests.util.pulsar.common.KeyedPulsarPartitionDataWriter; + +import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.common.util.Murmur3_32Hash; + +import static java.util.Collections.singletonList; +import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.RANGE_SIZE; + +/** We would consume from test splits by using {@link SubscriptionType#Key_Shared} subscription. */ +public class KeySharedSubscriptionContext extends MultipleTopicConsumingContext { + + private final String keyToRead; + private final String keyToExclude; + + public KeySharedSubscriptionContext(PulsarTestEnvironment environment) { + super(environment); + + this.keyToRead = randomAlphabetic(8); + + // Make sure they have different hash code. + int readHash = keyHash(keyToRead); + String randomKey; + do { + randomKey = randomAlphabetic(8); + } while (keyHash(randomKey) == readHash); + this.keyToExclude = randomKey; + } + + @Override + public ExternalSystemSplitDataWriter createSourceSplitDataWriter( + TestingSourceSettings sourceSettings) { + String partitionName = generatePartitionName(); + return new KeyedPulsarPartitionDataWriter(operator, partitionName, keyToRead, keyToExclude); + } + + @Override + protected String displayName() { + return "consume message by Key_Shared"; + } + + @Override + protected void setSourceBuilder(PulsarSourceBuilder builder) { + int keyHash = keyHash(keyToRead); + TopicRange range = new TopicRange(keyHash, keyHash); + + builder.setRangeGenerator(new FixedRangeGenerator(singletonList(range))); + } + + @Override + protected String subscriptionName() { + return "pulsar-key-shared-subscription"; + } + + @Override + protected SubscriptionType subscriptionType() { + return SubscriptionType.Key_Shared; + } + + // This method is copied from Pulsar for calculating message key hash. + private int keyHash(String key) { + return Murmur3_32Hash.getInstance().makeHash(key.getBytes()) % RANGE_SIZE; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/MultipleTopicConsumingContext.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/SharedSubscriptionContext.java similarity index 59% rename from flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/MultipleTopicConsumingContext.java rename to flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/SharedSubscriptionContext.java index 57027f33e1b5d..3e0730b8f59d3 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/MultipleTopicConsumingContext.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/SharedSubscriptionContext.java @@ -16,43 +16,32 @@ * limitations under the License. */ -package org.apache.flink.connector.pulsar.testutils.cases; +package org.apache.flink.tests.util.pulsar.source; import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; +import org.apache.flink.connector.pulsar.testutils.source.cases.MultipleTopicConsumingContext; import org.apache.pulsar.client.api.SubscriptionType; -import java.net.URL; -import java.util.Collections; -import java.util.List; +/** We would consume from test splits by using {@link SubscriptionType#Shared} subscription. */ +public class SharedSubscriptionContext extends MultipleTopicConsumingContext { -/** - * Pulsar external context that will create multiple topics with only one partitions as source - * splits. - */ -public class MultipleTopicConsumingContext extends MultipleTopicTemplateContext { - - public MultipleTopicConsumingContext(PulsarTestEnvironment environment) { - this(environment, Collections.emptyList()); - } - - public MultipleTopicConsumingContext( - PulsarTestEnvironment environment, List connectorJarPaths) { - super(environment, connectorJarPaths); + public SharedSubscriptionContext(PulsarTestEnvironment environment) { + super(environment); } @Override protected String displayName() { - return "consuming message on multiple topic"; + return "consume message by Shared"; } @Override protected String subscriptionName() { - return "flink-pulsar-multiple-topic-test"; + return "pulsar-shared-subscription"; } @Override protected SubscriptionType subscriptionType() { - return SubscriptionType.Exclusive; + return SubscriptionType.Shared; } } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-scala/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-scala/pom.xml index 0a514c8165e3e..beed64a76581e 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-scala/pom.xml +++ b/flink-end-to-end-tests/flink-end-to-end-tests-scala/pom.xml @@ -21,7 +21,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-file-sink-test/pom.xml b/flink-end-to-end-tests/flink-file-sink-test/pom.xml index 740d57e9bdb88..0a06535215ca9 100644 --- a/flink-end-to-end-tests/flink-file-sink-test/pom.xml +++ b/flink-end-to-end-tests/flink-file-sink-test/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-glue-schema-registry-avro-test/pom.xml b/flink-end-to-end-tests/flink-glue-schema-registry-avro-test/pom.xml index 96316403fdcea..c747c12d7509e 100644 --- a/flink-end-to-end-tests/flink-glue-schema-registry-avro-test/pom.xml +++ b/flink-end-to-end-tests/flink-glue-schema-registry-avro-test/pom.xml @@ -23,7 +23,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-end-to-end-tests/flink-glue-schema-registry-json-test/pom.xml b/flink-end-to-end-tests/flink-glue-schema-registry-json-test/pom.xml index 19121084be082..1633406271e37 100644 --- a/flink-end-to-end-tests/flink-glue-schema-registry-json-test/pom.xml +++ b/flink-end-to-end-tests/flink-glue-schema-registry-json-test/pom.xml @@ -23,7 +23,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-end-to-end-tests/flink-heavy-deployment-stress-test/pom.xml b/flink-end-to-end-tests/flink-heavy-deployment-stress-test/pom.xml index a96caf2c66cf6..77c2c7c46800d 100644 --- a/flink-end-to-end-tests/flink-heavy-deployment-stress-test/pom.xml +++ b/flink-end-to-end-tests/flink-heavy-deployment-stress-test/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-end-to-end-tests - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-high-parallelism-iterations-test/pom.xml b/flink-end-to-end-tests/flink-high-parallelism-iterations-test/pom.xml index f49e254609c1c..fbc2caefd23ed 100644 --- a/flink-end-to-end-tests/flink-high-parallelism-iterations-test/pom.xml +++ b/flink-end-to-end-tests/flink-high-parallelism-iterations-test/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-end-to-end-tests - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-local-recovery-and-allocation-test/pom.xml b/flink-end-to-end-tests/flink-local-recovery-and-allocation-test/pom.xml index 75bc71ec21168..592ed4a90a073 100644 --- a/flink-end-to-end-tests/flink-local-recovery-and-allocation-test/pom.xml +++ b/flink-end-to-end-tests/flink-local-recovery-and-allocation-test/pom.xml @@ -23,7 +23,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-metrics-availability-test/pom.xml b/flink-end-to-end-tests/flink-metrics-availability-test/pom.xml index e292694be3aba..a1819998053de 100644 --- a/flink-end-to-end-tests/flink-metrics-availability-test/pom.xml +++ b/flink-end-to-end-tests/flink-metrics-availability-test/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml index a28039b5a0468..f292ff3702366 100644 --- a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml +++ b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-netty-shuffle-memory-control-test/pom.xml b/flink-end-to-end-tests/flink-netty-shuffle-memory-control-test/pom.xml index 43e4737888259..752bc2dae7f0e 100644 --- a/flink-end-to-end-tests/flink-netty-shuffle-memory-control-test/pom.xml +++ b/flink-end-to-end-tests/flink-netty-shuffle-memory-control-test/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-end-to-end-tests - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-parent-child-classloading-test-lib-package/pom.xml b/flink-end-to-end-tests/flink-parent-child-classloading-test-lib-package/pom.xml index 2627dacd5a4c8..7198b00d29a10 100644 --- a/flink-end-to-end-tests/flink-parent-child-classloading-test-lib-package/pom.xml +++ b/flink-end-to-end-tests/flink-parent-child-classloading-test-lib-package/pom.xml @@ -29,7 +29,7 @@ flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-parent-child-classloading-test-program/pom.xml b/flink-end-to-end-tests/flink-parent-child-classloading-test-program/pom.xml index fc7f2f5bacce1..c045f4284e0d3 100644 --- a/flink-end-to-end-tests/flink-parent-child-classloading-test-program/pom.xml +++ b/flink-end-to-end-tests/flink-parent-child-classloading-test-program/pom.xml @@ -29,7 +29,7 @@ flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/pom.xml b/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/pom.xml index ecdc27ea3e287..a8bbce94e615c 100644 --- a/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/pom.xml +++ b/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/pom.xml @@ -23,7 +23,7 @@ under the License. flink-plugins-test org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-plugins-test/dummy-fs/pom.xml b/flink-end-to-end-tests/flink-plugins-test/dummy-fs/pom.xml index b7661dd6f67d9..93e34f2fa7d9b 100644 --- a/flink-end-to-end-tests/flink-plugins-test/dummy-fs/pom.xml +++ b/flink-end-to-end-tests/flink-plugins-test/dummy-fs/pom.xml @@ -23,7 +23,7 @@ under the License. flink-plugins-test org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-plugins-test/pom.xml b/flink-end-to-end-tests/flink-plugins-test/pom.xml index fc7c6a80a0e89..abc2a433ed392 100644 --- a/flink-end-to-end-tests/flink-plugins-test/pom.xml +++ b/flink-end-to-end-tests/flink-plugins-test/pom.xml @@ -25,7 +25,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 pom 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 432ea5eb15979..67a883dc54221 100644 --- a/flink-end-to-end-tests/flink-python-test/pom.xml +++ b/flink-end-to-end-tests/flink-python-test/pom.xml @@ -23,7 +23,7 @@ flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 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 421f9d2ac3237..32a3dc5adff8e 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 @@ -38,7 +38,7 @@ def word_count(): # used to test pipeline.jars and pipeline.classpaths config_key = sys.argv[1] config_value = sys.argv[2] - t_env.get_config().get_configuration().set_string(config_key, config_value) + t_env.get_config().set(config_key, config_value) # register Results table in table environment tmp_dir = tempfile.gettempdir() diff --git a/flink-end-to-end-tests/flink-python-test/src/main/java/org/apache/flink/python/tests/BatchPythonUdfSqlJob.java b/flink-end-to-end-tests/flink-python-test/src/main/java/org/apache/flink/python/tests/BatchPythonUdfSqlJob.java index 5639acaffa8ea..e8d286cb1d2f8 100644 --- a/flink-end-to-end-tests/flink-python-test/src/main/java/org/apache/flink/python/tests/BatchPythonUdfSqlJob.java +++ b/flink-end-to-end-tests/flink-python-test/src/main/java/org/apache/flink/python/tests/BatchPythonUdfSqlJob.java @@ -32,7 +32,7 @@ public class BatchPythonUdfSqlJob { public static void main(String[] args) { TableEnvironment tEnv = TableEnvironment.create(EnvironmentSettings.inBatchMode()); - tEnv.getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); + tEnv.getConfig().set(CoreOptions.DEFAULT_PARALLELISM, 1); tEnv.executeSql( "create temporary system function add_one as 'add_one.add_one' language python"); diff --git a/flink-end-to-end-tests/flink-queryable-state-test/pom.xml b/flink-end-to-end-tests/flink-queryable-state-test/pom.xml index 8eac23f213f5d..26c2988d43cf4 100644 --- a/flink-end-to-end-tests/flink-queryable-state-test/pom.xml +++ b/flink-end-to-end-tests/flink-queryable-state-test/pom.xml @@ -23,7 +23,7 @@ flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-quickstart-test/pom.xml b/flink-end-to-end-tests/flink-quickstart-test/pom.xml index 634c1851fffc7..2c91ff5f1dc25 100644 --- a/flink-end-to-end-tests/flink-quickstart-test/pom.xml +++ b/flink-end-to-end-tests/flink-quickstart-test/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-end-to-end-tests - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-rocksdb-state-memory-control-test/pom.xml b/flink-end-to-end-tests/flink-rocksdb-state-memory-control-test/pom.xml index b96df95c0f082..6425db3197ccd 100644 --- a/flink-end-to-end-tests/flink-rocksdb-state-memory-control-test/pom.xml +++ b/flink-end-to-end-tests/flink-rocksdb-state-memory-control-test/pom.xml @@ -23,7 +23,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-end-to-end-tests/flink-sql-client-test/pom.xml b/flink-end-to-end-tests/flink-sql-client-test/pom.xml index 8a6233c0108cb..77e4745d423e0 100644 --- a/flink-end-to-end-tests/flink-sql-client-test/pom.xml +++ b/flink-end-to-end-tests/flink-sql-client-test/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-state-evolution-test/pom.xml b/flink-end-to-end-tests/flink-state-evolution-test/pom.xml index 0e575997a2091..8e8f482cd6ca8 100644 --- a/flink-end-to-end-tests/flink-state-evolution-test/pom.xml +++ b/flink-end-to-end-tests/flink-state-evolution-test/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-stream-sql-test/pom.xml b/flink-end-to-end-tests/flink-stream-sql-test/pom.xml index a34ac2298cbbb..cf4612154c66f 100644 --- a/flink-end-to-end-tests/flink-stream-sql-test/pom.xml +++ b/flink-end-to-end-tests/flink-stream-sql-test/pom.xml @@ -24,7 +24,7 @@ flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/pom.xml b/flink-end-to-end-tests/flink-stream-state-ttl-test/pom.xml index 516867f0b054d..09ab468d82ca5 100644 --- a/flink-end-to-end-tests/flink-stream-state-ttl-test/pom.xml +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-end-to-end-tests - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-stream-stateful-job-upgrade-test/pom.xml b/flink-end-to-end-tests/flink-stream-stateful-job-upgrade-test/pom.xml index 3eda420e72e58..c587270692b42 100644 --- a/flink-end-to-end-tests/flink-stream-stateful-job-upgrade-test/pom.xml +++ b/flink-end-to-end-tests/flink-stream-stateful-job-upgrade-test/pom.xml @@ -23,7 +23,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-streaming-kafka-test-base/pom.xml b/flink-end-to-end-tests/flink-streaming-kafka-test-base/pom.xml index 799450d0ff147..0701c02a2743b 100644 --- a/flink-end-to-end-tests/flink-streaming-kafka-test-base/pom.xml +++ b/flink-end-to-end-tests/flink-streaming-kafka-test-base/pom.xml @@ -23,7 +23,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-end-to-end-tests/flink-streaming-kafka-test/pom.xml b/flink-end-to-end-tests/flink-streaming-kafka-test/pom.xml index d94999415770e..1b0731c90edc1 100644 --- a/flink-end-to-end-tests/flink-streaming-kafka-test/pom.xml +++ b/flink-end-to-end-tests/flink-streaming-kafka-test/pom.xml @@ -23,7 +23,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-end-to-end-tests/flink-streaming-kinesis-test/pom.xml b/flink-end-to-end-tests/flink-streaming-kinesis-test/pom.xml index c6cefd8f7ae05..e3e4754d9c9f3 100644 --- a/flink-end-to-end-tests/flink-streaming-kinesis-test/pom.xml +++ b/flink-end-to-end-tests/flink-streaming-kinesis-test/pom.xml @@ -23,7 +23,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 @@ -88,13 +88,6 @@ under the License. test-jar - - org.apache.flink - flink-connector-aws-base - ${project.version} - test - - org.junit.jupiter junit-jupiter diff --git a/flink-end-to-end-tests/flink-tpcds-test/pom.xml b/flink-end-to-end-tests/flink-tpcds-test/pom.xml index dbcfb2ecb0f3a..773e3694ae6e6 100644 --- a/flink-end-to-end-tests/flink-tpcds-test/pom.xml +++ b/flink-end-to-end-tests/flink-tpcds-test/pom.xml @@ -21,7 +21,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-end-to-end-tests/flink-tpcds-test/src/main/java/org/apache/flink/table/tpcds/TpcdsTestProgram.java b/flink-end-to-end-tests/flink-tpcds-test/src/main/java/org/apache/flink/table/tpcds/TpcdsTestProgram.java index c6251aaf85427..7c40ae2b1c79a 100644 --- a/flink-end-to-end-tests/flink-tpcds-test/src/main/java/org/apache/flink/table/tpcds/TpcdsTestProgram.java +++ b/flink-end-to-end-tests/flink-tpcds-test/src/main/java/org/apache/flink/table/tpcds/TpcdsTestProgram.java @@ -140,18 +140,14 @@ private static TableEnvironment prepareTableEnv(String sourceTablePath, Boolean // config Optimizer parameters // TODO use the default shuffle mode of batch runtime mode once FLINK-23470 is implemented tEnv.getConfig() - .getConfiguration() - .setString( + .set( ExecutionConfigOptions.TABLE_EXEC_SHUFFLE_MODE, GlobalStreamExchangeMode.POINTWISE_EDGES_PIPELINED.toString()); tEnv.getConfig() - .getConfiguration() - .setLong( + .set( OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, - 10 * 1024 * 1024); - tEnv.getConfig() - .getConfiguration() - .setBoolean(OptimizerConfigOptions.TABLE_OPTIMIZER_JOIN_REORDER_ENABLED, true); + 10 * 1024 * 1024L); + tEnv.getConfig().set(OptimizerConfigOptions.TABLE_OPTIMIZER_JOIN_REORDER_ENABLED, true); // register TPC-DS tables TPCDS_TABLES.forEach( diff --git a/flink-end-to-end-tests/flink-tpch-test/pom.xml b/flink-end-to-end-tests/flink-tpch-test/pom.xml index b534bded19f5a..a747749155966 100644 --- a/flink-end-to-end-tests/flink-tpch-test/pom.xml +++ b/flink-end-to-end-tests/flink-tpch-test/pom.xml @@ -21,7 +21,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml index 6a28b30d7240f..52cad76871dc4 100644 --- a/flink-end-to-end-tests/pom.xml +++ b/flink-end-to-end-tests/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/test-scripts/common_docker.sh b/flink-end-to-end-tests/test-scripts/common_docker.sh index 44040eee52a48..cd97d81419688 100644 --- a/flink-end-to-end-tests/test-scripts/common_docker.sh +++ b/flink-end-to-end-tests/test-scripts/common_docker.sh @@ -48,7 +48,7 @@ function build_image() { local server_pid=$! echo "Preparing Dockeriles" - retry_times_with_exponential_backoff 5 git clone https://github.com/apache/flink-docker.git --branch dev-master --single-branch + retry_times_with_exponential_backoff 5 git clone https://github.com/apache/flink-docker.git --branch dev-1.15 --single-branch local java_version=8 if [[ ${PROFILE} == *"jdk11"* ]]; then diff --git a/flink-examples/flink-examples-batch/pom.xml b/flink-examples/flink-examples-batch/pom.xml index 6ead88d94d278..cdc3c8c5955f7 100644 --- a/flink-examples/flink-examples-batch/pom.xml +++ b/flink-examples/flink-examples-batch/pom.xml @@ -24,7 +24,7 @@ under the License. org.apache.flink flink-examples - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/pom.xml b/flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/pom.xml index cf5824c62e94d..cb4e8846b5537 100644 --- a/flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/pom.xml +++ b/flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/pom.xml @@ -24,7 +24,7 @@ under the License. flink-examples-build-helper org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-examples/flink-examples-build-helper/flink-examples-streaming-state-machine/pom.xml b/flink-examples/flink-examples-build-helper/flink-examples-streaming-state-machine/pom.xml index bbf565be029bb..5c49245e4c9d5 100644 --- a/flink-examples/flink-examples-build-helper/flink-examples-streaming-state-machine/pom.xml +++ b/flink-examples/flink-examples-build-helper/flink-examples-streaming-state-machine/pom.xml @@ -24,7 +24,7 @@ under the License. flink-examples-build-helper org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-examples/flink-examples-build-helper/pom.xml b/flink-examples/flink-examples-build-helper/pom.xml index 5ade20d32bb6a..cb73913447d00 100644 --- a/flink-examples/flink-examples-build-helper/pom.xml +++ b/flink-examples/flink-examples-build-helper/pom.xml @@ -24,7 +24,7 @@ under the License. flink-examples org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-examples/flink-examples-streaming/pom.xml b/flink-examples/flink-examples-streaming/pom.xml index 744463d422f7c..24ac544ca317d 100644 --- a/flink-examples/flink-examples-streaming/pom.xml +++ b/flink-examples/flink-examples-streaming/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-examples - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-examples/flink-examples-table/pom.xml b/flink-examples/flink-examples-table/pom.xml index 1c09ec52ad1b4..d874aff840405 100644 --- a/flink-examples/flink-examples-table/pom.xml +++ b/flink-examples/flink-examples-table/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-examples - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-examples/flink-examples-table/src/test/java/org/apache/flink/table/examples/java/functions/AdvancedFunctionsExampleITCase.java b/flink-examples/flink-examples-table/src/test/java/org/apache/flink/table/examples/java/functions/AdvancedFunctionsExampleITCase.java index 4bc98c2e11b20..3306911da9ead 100644 --- a/flink-examples/flink-examples-table/src/test/java/org/apache/flink/table/examples/java/functions/AdvancedFunctionsExampleITCase.java +++ b/flink-examples/flink-examples-table/src/test/java/org/apache/flink/table/examples/java/functions/AdvancedFunctionsExampleITCase.java @@ -41,41 +41,41 @@ private void testExecuteLastDatedValueFunction(String consoleOutput) { assertThat( consoleOutput, containsString( - "| Guillermo Smith | (5,2020-12-05) |")); + "| Guillermo Smith | (5, 2020-12-05) |")); assertThat( consoleOutput, containsString( - "| John Turner | (12,2020-10-02) |")); + "| John Turner | (12, 2020-10-02) |")); assertThat( consoleOutput, containsString( - "| Brandy Sanders | (1,2020-10-14) |")); + "| Brandy Sanders | (1, 2020-10-14) |")); assertThat( consoleOutput, containsString( - "| Valeria Mendoza | (10,2020-06-02) |")); + "| Valeria Mendoza | (10, 2020-06-02) |")); assertThat( consoleOutput, containsString( - "| Ellen Ortega | (100,2020-06-18) |")); + "| Ellen Ortega | (100, 2020-06-18) |")); assertThat( consoleOutput, containsString( - "| Leann Holloway | (9,2020-05-26) |")); + "| Leann Holloway | (9, 2020-05-26) |")); } private void testExecuteInternalRowMergerFunction(String consoleOutput) { assertThat( consoleOutput, containsString( - "| Guillermo Smith | (1992-12-12,New Jersey,816-... |")); + "| Guillermo Smith | (1992-12-12, New Jersey, 81... |")); assertThat( consoleOutput, containsString( - "| Valeria Mendoza | (1970-03-28,Los Angeles,928... |")); + "| Valeria Mendoza | (1970-03-28, Los Angeles, 9... |")); assertThat( consoleOutput, containsString( - "| Leann Holloway | (1989-05-21,Eugene,614-889-... |")); + "| Leann Holloway | (1989-05-21, Eugene, 614-88... |")); } } diff --git a/flink-examples/pom.xml b/flink-examples/pom.xml index 8c89630c107fc..ab51ec0a87d39 100644 --- a/flink-examples/pom.xml +++ b/flink-examples/pom.xml @@ -24,7 +24,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-external-resources/flink-external-resource-gpu/pom.xml b/flink-external-resources/flink-external-resource-gpu/pom.xml index 9416ec12e76f5..e045b7227e485 100644 --- a/flink-external-resources/flink-external-resource-gpu/pom.xml +++ b/flink-external-resources/flink-external-resource-gpu/pom.xml @@ -25,7 +25,7 @@ under the License. flink-external-resources org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-external-resources/pom.xml b/flink-external-resources/pom.xml index 4a98561c9847b..99520497dc8f3 100644 --- a/flink-external-resources/pom.xml +++ b/flink-external-resources/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-filesystems/flink-azure-fs-hadoop/pom.xml b/flink-filesystems/flink-azure-fs-hadoop/pom.xml index edcd5d2aabd3d..78ab0898d3b80 100644 --- a/flink-filesystems/flink-azure-fs-hadoop/pom.xml +++ b/flink-filesystems/flink-azure-fs-hadoop/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-filesystems - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-filesystems/flink-azure-fs-hadoop/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-azure-fs-hadoop/src/main/resources/META-INF/NOTICE index c2a59b5105ae2..63516ed042709 100644 --- a/flink-filesystems/flink-azure-fs-hadoop/src/main/resources/META-INF/NOTICE +++ b/flink-filesystems/flink-azure-fs-hadoop/src/main/resources/META-INF/NOTICE @@ -6,7 +6,7 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) -- com.fasterxml.jackson.core:jackson-core:2.13.0 +- com.fasterxml.jackson.core:jackson-core:2.13.2 - com.google.errorprone:error_prone_annotations:2.2.0 - com.google.guava:failureaccess:1.0 - com.google.guava:guava:27.0-jre diff --git a/flink-filesystems/flink-fs-hadoop-shaded/pom.xml b/flink-filesystems/flink-fs-hadoop-shaded/pom.xml index 8f8a70c98817a..6d36614adb3c5 100644 --- a/flink-filesystems/flink-fs-hadoop-shaded/pom.xml +++ b/flink-filesystems/flink-fs-hadoop-shaded/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-filesystems - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE index e55f90db980cd..39269ee8e1751 100644 --- a/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE +++ b/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE @@ -22,9 +22,9 @@ This project bundles the following dependencies under the Apache Software Licens - com.google.guava:guava:27.0-jre - com.google.guava:listenablefuture:9999.0-empty-to-avoid-conflict-with-guava - com.google.j2objc:j2objc-annotations:1.1 -- com.fasterxml.jackson.core:jackson-annotations:2.13.0 -- com.fasterxml.jackson.core:jackson-core:2.13.0 -- com.fasterxml.jackson.core:jackson-databind:2.13.0 +- com.fasterxml.jackson.core:jackson-annotations:2.13.2 +- com.fasterxml.jackson.core:jackson-core:2.13.2 +- com.fasterxml.jackson.core:jackson-databind:2.13.2.2 - com.fasterxml.woodstox:woodstox-core:5.0.3 This project bundles the following dependencies under the MIT (https://opensource.org/licenses/MIT) diff --git a/flink-filesystems/flink-gs-fs-hadoop/pom.xml b/flink-filesystems/flink-gs-fs-hadoop/pom.xml index ce00f7a9864a2..e443d1532e89d 100644 --- a/flink-filesystems/flink-gs-fs-hadoop/pom.xml +++ b/flink-filesystems/flink-gs-fs-hadoop/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-filesystems - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-filesystems/flink-gs-fs-hadoop/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-gs-fs-hadoop/src/main/resources/META-INF/NOTICE index fd302700df886..aa1afa70fa786 100644 --- a/flink-filesystems/flink-gs-fs-hadoop/src/main/resources/META-INF/NOTICE +++ b/flink-filesystems/flink-gs-fs-hadoop/src/main/resources/META-INF/NOTICE @@ -6,7 +6,7 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) -- com.fasterxml.jackson.core:jackson-core:2.13.0 +- com.fasterxml.jackson.core:jackson-core:2.13.2 - com.google.android:annotations:4.1.1.4 - com.google.api-client:google-api-client-jackson2:1.32.2 - com.google.api-client:google-api-client:1.33.0 diff --git a/flink-filesystems/flink-hadoop-fs/pom.xml b/flink-filesystems/flink-hadoop-fs/pom.xml index c9a73dd6376c0..64cab658112f3 100644 --- a/flink-filesystems/flink-hadoop-fs/pom.xml +++ b/flink-filesystems/flink-hadoop-fs/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-filesystems - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-filesystems/flink-oss-fs-hadoop/pom.xml b/flink-filesystems/flink-oss-fs-hadoop/pom.xml index fba791dd2ee04..bbf95f95e92c0 100644 --- a/flink-filesystems/flink-oss-fs-hadoop/pom.xml +++ b/flink-filesystems/flink-oss-fs-hadoop/pom.xml @@ -21,7 +21,7 @@ under the License. flink-filesystems org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-filesystems/flink-s3-fs-base/pom.xml b/flink-filesystems/flink-s3-fs-base/pom.xml index 9ccf84ed08542..d3f3941f4b040 100644 --- a/flink-filesystems/flink-s3-fs-base/pom.xml +++ b/flink-filesystems/flink-s3-fs-base/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-filesystems - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/HAJobRunOnMinioS3StoreITCase.java b/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/HAJobRunOnMinioS3StoreITCase.java index bd34590d0eede..f53a801aa6fc2 100644 --- a/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/HAJobRunOnMinioS3StoreITCase.java +++ b/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/HAJobRunOnMinioS3StoreITCase.java @@ -18,7 +18,6 @@ package org.apache.flink.fs.s3.common; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.testutils.AllCallbackWrapper; @@ -39,7 +38,6 @@ import org.junit.jupiter.api.Order; import org.junit.jupiter.api.extension.RegisterExtension; -import java.time.Duration; import java.util.List; import static org.apache.flink.shaded.guava30.com.google.common.base.Predicates.not; @@ -130,9 +128,7 @@ protected void runAfterJobTermination() throws Exception { FileSystemJobResultStore ::hasValidDirtyJobResultStoreEntryExtension); }, - Deadline.fromNow(Duration.ofSeconds(60)), - 2000L, - "Wait for the JobResult being written to the JobResultStore."); + 2000L); final S3ObjectSummary objRef = Iterables.getOnlyElement(getObjectsFromJobResultStore()); assertThat(objRef.getKey()) diff --git a/flink-filesystems/flink-s3-fs-hadoop/pom.xml b/flink-filesystems/flink-s3-fs-hadoop/pom.xml index 6c3799d8fadde..0dd5e3a566e0c 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/pom.xml +++ b/flink-filesystems/flink-s3-fs-hadoop/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-filesystems - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-s3-fs-hadoop/src/main/resources/META-INF/NOTICE index 02d7e43585a3f..a38114f6ef79f 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/src/main/resources/META-INF/NOTICE +++ b/flink-filesystems/flink-s3-fs-hadoop/src/main/resources/META-INF/NOTICE @@ -9,10 +9,10 @@ This project bundles the following dependencies under the Apache Software Licens - com.amazonaws:aws-java-sdk-s3:1.11.951 - com.amazonaws:aws-java-sdk-sts:1.11.951 - com.amazonaws:jmespath-java:1.11.951 -- com.fasterxml.jackson.core:jackson-annotations:2.13.0 -- com.fasterxml.jackson.core:jackson-core:2.13.0 -- com.fasterxml.jackson.core:jackson-databind:2.13.0 -- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.0 +- com.fasterxml.jackson.core:jackson-annotations:2.13.2 +- com.fasterxml.jackson.core:jackson-core:2.13.2 +- com.fasterxml.jackson.core:jackson-databind:2.13.2.2 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.2 - com.fasterxml.woodstox:woodstox-core:5.0.3 - com.google.errorprone:error_prone_annotations:2.2.0 - com.google.guava:failureaccess:1.0 diff --git a/flink-filesystems/flink-s3-fs-presto/pom.xml b/flink-filesystems/flink-s3-fs-presto/pom.xml index 9a876559f65e7..cdcefeb7cf29e 100644 --- a/flink-filesystems/flink-s3-fs-presto/pom.xml +++ b/flink-filesystems/flink-s3-fs-presto/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-filesystems - 1.15-SNAPSHOT + 1.15.0 .. @@ -33,7 +33,7 @@ under the License. jar - 0.257 + 0.272 diff --git a/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE index 65ee90d99e0b0..ebcbf9a40f326 100644 --- a/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE +++ b/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE @@ -17,15 +17,15 @@ This project bundles the following dependencies under the Apache Software Licens - com.amazonaws:aws-java-sdk-s3:1.11.951 - com.amazonaws:aws-java-sdk-sts:1.11.951 - com.amazonaws:jmespath-java:1.11.951 -- com.facebook.presto:presto-common:0.257 -- com.facebook.presto:presto-hive:0.257 -- com.facebook.presto:presto-hive-common:0.257 -- com.facebook.presto:presto-hive-metastore:0.257 +- com.facebook.presto:presto-common:0.272 +- com.facebook.presto:presto-hive:0.272 +- com.facebook.presto:presto-hive-common:0.272 +- com.facebook.presto:presto-hive-metastore:0.272 - com.facebook.presto.hadoop:hadoop-apache2:2.7.4-9 -- com.fasterxml.jackson.core:jackson-annotations:2.13.0 -- com.fasterxml.jackson.core:jackson-core:2.13.0 -- com.fasterxml.jackson.core:jackson-databind:2.13.0 -- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.0 +- com.fasterxml.jackson.core:jackson-annotations:2.13.2 +- com.fasterxml.jackson.core:jackson-core:2.13.2 +- com.fasterxml.jackson.core:jackson-databind:2.13.2.2 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.2 - com.fasterxml.woodstox:woodstox-core:5.0.3 - com.google.guava:guava:26.0-jre - com.google.inject:guice:4.2.2 @@ -35,7 +35,7 @@ This project bundles the following dependencies under the Apache Software Licens - io.airlift:units:1.3 - io.airlift:slice:0.38 - joda-time:joda-time:2.5 -- org.alluxio:alluxio-shaded-client:2.5.0-3 +- org.alluxio:alluxio-shaded-client:2.7.3 - org.apache.commons:commons-configuration2:2.1.1 - org.apache.commons:commons-lang3:3.3.2 - org.apache.commons:commons-text:1.4 @@ -46,6 +46,7 @@ This project bundles the following dependencies under the Apache Software Licens - org.apache.htrace:htrace-core4:4.1.0-incubating - org.apache.httpcomponents:httpclient:4.5.13 - org.apache.httpcomponents:httpcore:4.4.14 +- org.apache.hudi:hudi-presto-bundle:0.10.1 - org.weakref:jmxutils:1.19 - software.amazon.ion:ion-java:1.0.2 diff --git a/flink-filesystems/pom.xml b/flink-filesystems/pom.xml index fdd61ab9c43ba..37ecbe0aa5ffb 100644 --- a/flink-filesystems/pom.xml +++ b/flink-filesystems/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/flink-avro-confluent-registry/pom.xml b/flink-formats/flink-avro-confluent-registry/pom.xml index 4b791287a44c1..d564d2a29c2c5 100644 --- a/flink-formats/flink-avro-confluent-registry/pom.xml +++ b/flink-formats/flink-avro-confluent-registry/pom.xml @@ -23,7 +23,7 @@ under the License. flink-formats org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-formats/flink-avro-glue-schema-registry/pom.xml b/flink-formats/flink-avro-glue-schema-registry/pom.xml index 41e59016cf1e3..5ba84a7cc0e6f 100644 --- a/flink-formats/flink-avro-glue-schema-registry/pom.xml +++ b/flink-formats/flink-avro-glue-schema-registry/pom.xml @@ -23,7 +23,7 @@ under the License. flink-formats org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-formats/flink-avro/pom.xml b/flink-formats/flink-avro/pom.xml index 1f60cee550ce2..c8ea0bb090328 100644 --- a/flink-formats/flink-avro/pom.xml +++ b/flink-formats/flink-avro/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/flink-compress/pom.xml b/flink-formats/flink-compress/pom.xml index dae4a8b387fca..e1c1f0f80c5a7 100644 --- a/flink-formats/flink-compress/pom.xml +++ b/flink-formats/flink-compress/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/flink-csv/pom.xml b/flink-formats/flink-csv/pom.xml index d2fbd4743384a..b46613fa44079 100644 --- a/flink-formats/flink-csv/pom.xml +++ b/flink-formats/flink-csv/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowDataDeserializationSchema.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowDataDeserializationSchema.java index 2d58007a0b0e6..7dfc5650c99c4 100644 --- a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowDataDeserializationSchema.java +++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowDataDeserializationSchema.java @@ -72,7 +72,7 @@ private CsvRowDataDeserializationSchema( this.resultTypeInfo = resultTypeInfo; this.runtimeConverter = new CsvToRowDataConverters(ignoreParseErrors).createRowConverter(rowType, true); - this.csvSchema = CsvRowSchemaConverter.convert(rowType); + this.csvSchema = csvSchema; this.objectReader = new CsvMapper().readerFor(JsonNode.class).with(csvSchema); this.ignoreParseErrors = ignoreParseErrors; } diff --git a/flink-formats/flink-format-common/pom.xml b/flink-formats/flink-format-common/pom.xml index ca31700a9e988..d6a31a490f471 100644 --- a/flink-formats/flink-format-common/pom.xml +++ b/flink-formats/flink-format-common/pom.xml @@ -22,7 +22,7 @@ under the License. flink-formats org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-formats/flink-hadoop-bulk/pom.xml b/flink-formats/flink-hadoop-bulk/pom.xml index 60abe050c21aa..cea562ddf5a11 100644 --- a/flink-formats/flink-hadoop-bulk/pom.xml +++ b/flink-formats/flink-hadoop-bulk/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/flink-json-glue-schema-registry/pom.xml b/flink-formats/flink-json-glue-schema-registry/pom.xml index 3982ae53d2dbf..c0755cf391179 100644 --- a/flink-formats/flink-json-glue-schema-registry/pom.xml +++ b/flink-formats/flink-json-glue-schema-registry/pom.xml @@ -23,7 +23,7 @@ under the License. flink-formats org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-formats/flink-json/pom.xml b/flink-formats/flink-json/pom.xml index a04294917a1d8..c6c5cd11850a1 100644 --- a/flink-formats/flink-json/pom.xml +++ b/flink-formats/flink-json/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/flink-orc-nohive/pom.xml b/flink-formats/flink-orc-nohive/pom.xml index a619d610ddf8e..34431207ea5d2 100644 --- a/flink-formats/flink-orc-nohive/pom.xml +++ b/flink-formats/flink-orc-nohive/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/flink-orc/pom.xml b/flink-formats/flink-orc/pom.xml index bf5a380a828b1..bbaf5ff01c406 100644 --- a/flink-formats/flink-orc/pom.xml +++ b/flink-formats/flink-orc/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. @@ -45,12 +45,6 @@ under the License. provided - - org.apache.flink - flink-connector-base - ${project.version} - - @@ -210,33 +204,6 @@ under the License. - - - org.apache.maven.plugins - maven-shade-plugin - - - shade-flink - package - - shade - - - - - org.apache.flink:flink-connector-base - - - - - org.apache.flink.connector.base - org.apache.flink.format.orc.shaded.org.apache.flink.connector.base - - - - - - diff --git a/flink-formats/flink-parquet/pom.xml b/flink-formats/flink-parquet/pom.xml index 8efa5a2c6824b..47662b1df23d2 100644 --- a/flink-formats/flink-parquet/pom.xml +++ b/flink-formats/flink-parquet/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. @@ -45,12 +45,6 @@ under the License. provided - - org.apache.flink - flink-connector-base - ${project.version} - - @@ -314,33 +308,6 @@ under the License. - - - org.apache.maven.plugins - maven-shade-plugin - - - shade-flink - package - - shade - - - - - org.apache.flink:flink-connector-base - - - - - org.apache.flink.connector.base - org.apache.flink.format.parquet.shaded.org.apache.flink.connector.base - - - - - - diff --git a/flink-formats/flink-sequence-file/pom.xml b/flink-formats/flink-sequence-file/pom.xml index b4391c500be79..ba7363db1b778 100644 --- a/flink-formats/flink-sequence-file/pom.xml +++ b/flink-formats/flink-sequence-file/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/flink-sql-avro-confluent-registry/pom.xml b/flink-formats/flink-sql-avro-confluent-registry/pom.xml index b00aa61a8a527..c14ae92c11b6c 100644 --- a/flink-formats/flink-sql-avro-confluent-registry/pom.xml +++ b/flink-formats/flink-sql-avro-confluent-registry/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE index bfddc9b0f8e80..48394cfcebf36 100644 --- a/flink-formats/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -7,9 +7,9 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) - org.apache.avro:avro:1.10.0 -- com.fasterxml.jackson.core:jackson-core:2.13.0 -- com.fasterxml.jackson.core:jackson-databind:2.13.0 -- com.fasterxml.jackson.core:jackson-annotations:2.13.0 +- com.fasterxml.jackson.core:jackson-core:2.13.2 +- com.fasterxml.jackson.core:jackson-databind:2.13.2.2 +- com.fasterxml.jackson.core:jackson-annotations:2.13.2 - org.apache.commons:commons-compress:1.21 - io.confluent:kafka-schema-registry-client:6.2.2 - org.apache.kafka:kafka-clients:6.2.2-ccs diff --git a/flink-formats/flink-sql-avro/pom.xml b/flink-formats/flink-sql-avro/pom.xml index fb2cdabe01fbd..e808b45468db8 100644 --- a/flink-formats/flink-sql-avro/pom.xml +++ b/flink-formats/flink-sql-avro/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE b/flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE index dcf6dcc9a540a..0096914beaca8 100644 --- a/flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE +++ b/flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE @@ -7,7 +7,7 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) - org.apache.avro:avro:1.10.0 -- com.fasterxml.jackson.core:jackson-core:2.13.0 -- com.fasterxml.jackson.core:jackson-databind:2.13.0 -- com.fasterxml.jackson.core:jackson-annotations:2.13.0 +- com.fasterxml.jackson.core:jackson-core:2.13.2 +- com.fasterxml.jackson.core:jackson-databind:2.13.2.2 +- com.fasterxml.jackson.core:jackson-annotations:2.13.2 - org.apache.commons:commons-compress:1.21 diff --git a/flink-formats/flink-sql-orc/pom.xml b/flink-formats/flink-sql-orc/pom.xml index dd26c3a5c9f5a..449b53bffe490 100644 --- a/flink-formats/flink-sql-orc/pom.xml +++ b/flink-formats/flink-sql-orc/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/flink-sql-parquet/pom.xml b/flink-formats/flink-sql-parquet/pom.xml index f857017d4bcc6..1b6bd8a1005c4 100644 --- a/flink-formats/flink-sql-parquet/pom.xml +++ b/flink-formats/flink-sql-parquet/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/pom.xml b/flink-formats/pom.xml index 44272566db40a..2e6be0f48015c 100644 --- a/flink-formats/pom.xml +++ b/flink-formats/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-fs-tests/pom.xml b/flink-fs-tests/pom.xml index 4ba5dc774b689..bee60d5d9d3fd 100644 --- a/flink-fs-tests/pom.xml +++ b/flink-fs-tests/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-java/pom.xml b/flink-java/pom.xml index ff789cc8593f6..9af518436334b 100644 --- a/flink-java/pom.xml +++ b/flink-java/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-kubernetes/pom.xml b/flink-kubernetes/pom.xml index 9173ebad36ff5..3f16f9724fd45 100644 --- a/flink-kubernetes/pom.xml +++ b/flink-kubernetes/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointIDCounter.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointIDCounter.java index a08146a956e0a..8eac11ec9cc1e 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointIDCounter.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointIDCounter.java @@ -25,6 +25,7 @@ import org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector; import org.apache.flink.runtime.checkpoint.CheckpointIDCounter; import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.concurrent.FutureUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -32,6 +33,7 @@ import javax.annotation.Nullable; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicLong; import static org.apache.flink.kubernetes.utils.Constants.CHECKPOINT_COUNTER_KEY; @@ -71,25 +73,32 @@ public void start() { } @Override - public void shutdown(JobStatus jobStatus) { + public CompletableFuture shutdown(JobStatus jobStatus) { if (!running) { - return; + return FutureUtils.completedVoidFuture(); } running = false; LOG.info("Shutting down."); if (jobStatus.isGloballyTerminalState()) { LOG.info("Removing counter from ConfigMap {}", configMapName); - kubeClient.checkAndUpdateConfigMap( - configMapName, - configMap -> { - if (isValidOperation(configMap)) { - configMap.getData().remove(CHECKPOINT_COUNTER_KEY); - return Optional.of(configMap); - } - return Optional.empty(); - }); + return kubeClient + .checkAndUpdateConfigMap( + configMapName, + configMap -> { + if (isValidOperation(configMap)) { + configMap.getData().remove(CHECKPOINT_COUNTER_KEY); + return Optional.of(configMap); + } + return Optional.empty(); + }) + // checkAndUpdateConfigMap only returns false if the callback returned an empty + // ConfigMap. We don't want to continue the cleanup in that case, i.e. we can + // ignore the return value + .thenApply(valueChanged -> null); } + + return FutureUtils.completedVoidFuture(); } private boolean isValidOperation(KubernetesConfigMap configMap) { diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointRecoveryFactory.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointRecoveryFactory.java index ea78ecbbd2829..7150034bbb8a5 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointRecoveryFactory.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointRecoveryFactory.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointIDCounter; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.state.SharedStateRegistryFactory; import javax.annotation.Nullable; @@ -81,7 +82,8 @@ public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobID, int maxNumberOfCheckpointsToRetain, SharedStateRegistryFactory sharedStateRegistryFactory, - Executor ioExecutor) + Executor ioExecutor, + RestoreMode restoreMode) throws Exception { final String configMapName = getConfigMapNameFunction.apply(jobID); KubernetesUtils.createConfigMapIfItDoesNotExist(kubeClient, configMapName, clusterId); @@ -94,7 +96,8 @@ public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( lockIdentity, maxNumberOfCheckpointsToRetain, sharedStateRegistryFactory, - ioExecutor); + ioExecutor, + restoreMode); } @Override diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStore.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStore.java index cc7153d16b79a..0716b58ec3489 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStore.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStore.java @@ -51,6 +51,7 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.function.Predicate; @@ -464,13 +465,14 @@ public Collection getAllHandles() throws Exception { * It returns the {@link RetrievableStateHandle} stored under the given state node if any. * * @param key Key to be removed from ConfigMap - * @return True if the state handle is removed successfully + * @return True if the state handle isn't listed anymore. * @throws Exception if removing the key or discarding the state failed */ @Override public boolean releaseAndTryRemove(String key) throws Exception { checkNotNull(key, "Key in ConfigMap."); final AtomicReference> stateHandleRefer = new AtomicReference<>(); + final AtomicBoolean stateHandleDoesNotExist = new AtomicBoolean(false); return updateConfigMap( configMap -> { final String content = configMap.getData().get(key); @@ -496,6 +498,8 @@ public boolean releaseAndTryRemove(String key) throws Exception { Objects.requireNonNull(configMap.getData().remove(key)); } return Optional.of(configMap); + } else { + stateHandleDoesNotExist.set(true); } return Optional.empty(); }) @@ -516,7 +520,8 @@ public boolean releaseAndTryRemove(String key) throws Exception { throw new CompletionException(e); } } - return CompletableFuture.completedFuture(updated); + return CompletableFuture.completedFuture( + stateHandleDoesNotExist.get() || updated); }) .get(); } diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java index e02d3a824e2f7..f8afbafde0aaa 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java @@ -37,6 +37,7 @@ import org.apache.flink.runtime.checkpoint.DefaultCompletedCheckpointStoreUtils; import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.jobmanager.DefaultJobGraphStore; import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.jobmanager.NoOpJobGraphStoreWatcher; @@ -296,6 +297,7 @@ public static KubernetesStateHandleStore createJobGraphStateHandleStor * @param lockIdentity lock identity to check the leadership * @param maxNumberOfCheckpointsToRetain max number of checkpoints to retain on state store * handle + * @param restoreMode the mode in which the job is restoring * @return a {@link DefaultCompletedCheckpointStore} with {@link KubernetesStateHandleStore}. * @throws Exception when create the storage helper failed */ @@ -307,7 +309,8 @@ public static CompletedCheckpointStore createCompletedCheckpointStore( @Nullable String lockIdentity, int maxNumberOfCheckpointsToRetain, SharedStateRegistryFactory sharedStateRegistryFactory, - Executor ioExecutor) + Executor ioExecutor, + RestoreMode restoreMode) throws Exception { final RetrievableStateStorageHelper stateStorage = @@ -331,7 +334,7 @@ public static CompletedCheckpointStore createCompletedCheckpointStore( stateHandleStore, KubernetesCheckpointStoreUtil.INSTANCE, checkpoints, - sharedStateRegistryFactory.create(ioExecutor, checkpoints), + sharedStateRegistryFactory.create(ioExecutor, checkpoints, restoreMode), executor); } diff --git a/flink-kubernetes/src/main/resources/META-INF/NOTICE b/flink-kubernetes/src/main/resources/META-INF/NOTICE index 1adda9dcd7562..08f96e407167f 100644 --- a/flink-kubernetes/src/main/resources/META-INF/NOTICE +++ b/flink-kubernetes/src/main/resources/META-INF/NOTICE @@ -6,11 +6,11 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) -- com.fasterxml.jackson.core:jackson-annotations:2.13.0 -- com.fasterxml.jackson.core:jackson-core:2.13.0 -- com.fasterxml.jackson.core:jackson-databind:2.13.0 -- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.13.0 -- com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.13.0 +- com.fasterxml.jackson.core:jackson-annotations:2.13.2 +- com.fasterxml.jackson.core:jackson-core:2.13.2 +- com.fasterxml.jackson.core:jackson-databind:2.13.2.2 +- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.13.2 +- com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.13.2 - com.squareup.okhttp3:logging-interceptor:3.14.9 - com.squareup.okhttp3:okhttp:3.14.9 - com.squareup.okio:okio:1.17.2 diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriverTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriverTest.java index 9063f549edf62..6d09f903ca04c 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriverTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriverTest.java @@ -18,7 +18,6 @@ package org.apache.flink.kubernetes; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; import org.apache.flink.kubernetes.configuration.KubernetesResourceManagerDriverConfiguration; @@ -40,7 +39,6 @@ import io.fabric8.kubernetes.api.model.ResourceRequirements; import org.junit.Test; -import java.time.Duration; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -211,12 +209,7 @@ public void testNewWatchCreationWhenKubernetesTooOldResourceVersionException() new KubernetesTooOldResourceVersionException( new Exception("too old resource version"))); // Verify the old watch is closed and a new one is created - CommonTestUtils.waitUntilCondition( - () -> getPodsWatches().size() == 2, - Deadline.fromNow(Duration.ofSeconds(TIMEOUT_SEC)), - String.format( - "New watch is not created in %s seconds.", - TIMEOUT_SEC)); + CommonTestUtils.waitUntilCondition(() -> getPodsWatches().size() == 2); assertThat(getPodsWatches().get(0).isClosed(), is(true)); assertThat(getPodsWatches().get(1).isClosed(), is(false)); }); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointIDCounterTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointIDCounterTest.java index d6d4313c62c78..f599e5c30d82b 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointIDCounterTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointIDCounterTest.java @@ -18,13 +18,19 @@ package org.apache.flink.kubernetes.highavailability; +import org.apache.flink.api.common.JobStatus; import org.apache.flink.core.testutils.FlinkMatchers; import org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector; +import org.apache.flink.kubernetes.utils.Constants; +import org.apache.flink.kubernetes.utils.KubernetesUtils; import org.junit.Test; +import java.util.concurrent.CompletionException; + import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.fail; /** Tests for {@link KubernetesCheckpointIDCounter} operations. */ @@ -50,6 +56,138 @@ public void testGetAndIncrement() throws Exception { }; } + @Test + public void testShutdown() throws Exception { + new Context() { + { + runTest( + () -> { + leaderCallbackGrantLeadership(); + + final KubernetesCheckpointIDCounter checkpointIDCounter = + new KubernetesCheckpointIDCounter( + flinkKubeClient, LEADER_CONFIGMAP_NAME, LOCK_IDENTITY); + checkpointIDCounter.start(); + + checkpointIDCounter.setCount(100L); + + assertThat( + getLeaderConfigMap() + .getData() + .get(Constants.CHECKPOINT_COUNTER_KEY), + is("100")); + + checkpointIDCounter.shutdown(JobStatus.FINISHED).join(); + + assertThat( + getLeaderConfigMap() + .getData() + .containsKey(Constants.CHECKPOINT_COUNTER_KEY), + is(false)); + }); + } + }; + } + + @Test + public void testShutdownForLocallyTerminatedJobStatus() throws Exception { + new Context() { + { + runTest( + () -> { + leaderCallbackGrantLeadership(); + + final KubernetesCheckpointIDCounter checkpointIDCounter = + new KubernetesCheckpointIDCounter( + flinkKubeClient, LEADER_CONFIGMAP_NAME, LOCK_IDENTITY); + checkpointIDCounter.start(); + + checkpointIDCounter.setCount(100L); + + assertThat( + getLeaderConfigMap() + .getData() + .get(Constants.CHECKPOINT_COUNTER_KEY), + is("100")); + + checkpointIDCounter.shutdown(JobStatus.SUSPENDED).join(); + + assertThat( + getLeaderConfigMap() + .getData() + .containsKey(Constants.CHECKPOINT_COUNTER_KEY), + is(true)); + }); + } + }; + } + + @Test + public void testIdempotentShutdown() throws Exception { + new Context() { + { + runTest( + () -> { + leaderCallbackGrantLeadership(); + + final KubernetesCheckpointIDCounter checkpointIDCounter = + new KubernetesCheckpointIDCounter( + flinkKubeClient, LEADER_CONFIGMAP_NAME, LOCK_IDENTITY); + checkpointIDCounter.start(); + + assertThat( + getLeaderConfigMap() + .getData() + .containsKey(Constants.CHECKPOINT_COUNTER_KEY), + is(false)); + + checkpointIDCounter.shutdown(JobStatus.FINISHED).join(); + + assertThat( + getLeaderConfigMap() + .getData() + .containsKey(Constants.CHECKPOINT_COUNTER_KEY), + is(false)); + + // a second shutdown should work without causing any errors + checkpointIDCounter.shutdown(JobStatus.FINISHED).join(); + }); + } + }; + } + + @Test + public void testShutdownFailureDueToMissingConfigMap() throws Exception { + new Context() { + { + runTest( + () -> { + leaderCallbackGrantLeadership(); + + final KubernetesCheckpointIDCounter checkpointIDCounter = + new KubernetesCheckpointIDCounter( + flinkKubeClient, LEADER_CONFIGMAP_NAME, LOCK_IDENTITY); + checkpointIDCounter.start(); + + // deleting the ConfigMap from outside of the CheckpointIDCounter while + // still using the counter (which is stored as an entry in the + // ConfigMap) causes an unexpected failure which we want to simulate + // here + flinkKubeClient.deleteConfigMap(LEADER_CONFIGMAP_NAME); + + assertThrows( + CompletionException.class, + () -> checkpointIDCounter.shutdown(JobStatus.FINISHED).get()); + + // fixing the internal issue should make the shutdown succeed again + KubernetesUtils.createConfigMapIfItDoesNotExist( + flinkKubeClient, LEADER_CONFIGMAP_NAME, getClusterId()); + checkpointIDCounter.shutdown(JobStatus.FINISHED).get(); + }); + } + }; + } + @Test public void testGetAndIncrementWithNoLeadership() throws Exception { new Context() { @@ -65,7 +203,7 @@ public void testGetAndIncrementWithNoLeadership() throws Exception { // lost leadership getLeaderCallback().notLeader(); - electionEventHandler.waitForRevokeLeader(TIMEOUT); + electionEventHandler.waitForRevokeLeader(); getLeaderConfigMap() .getAnnotations() .remove(KubernetesLeaderElector.LEADER_ANNOTATION_KEY); @@ -146,7 +284,7 @@ public void testSetWithNoLeadershipShouldNotBeIssued() throws Exception { // lost leadership getLeaderCallback().notLeader(); - electionEventHandler.waitForRevokeLeader(TIMEOUT); + electionEventHandler.waitForRevokeLeader(); getLeaderConfigMap() .getAnnotations() .remove(KubernetesLeaderElector.LEADER_ANNOTATION_KEY); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java index 4559d175cf280..baa0603df529a 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java @@ -25,7 +25,6 @@ import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.client.program.ClusterClient; @@ -59,7 +58,6 @@ import org.junit.rules.TemporaryFolder; import java.io.IOException; -import java.time.Duration; import java.util.Random; import java.util.concurrent.TimeUnit; @@ -110,7 +108,6 @@ public void testRecoverFromSavepoint() throws Exception { // Wait until all tasks running and getting a successful savepoint CommonTestUtils.waitUntilCondition( () -> triggerSavepoint(clusterClient, jobGraph.getJobID(), savepointPath) != null, - Deadline.fromNow(TestingUtils.infiniteDuration()), 1000); // Trigger savepoint 2 @@ -121,7 +118,6 @@ public void testRecoverFromSavepoint() throws Exception { clusterClient.cancel(jobGraph.getJobID()); CommonTestUtils.waitUntilCondition( () -> clusterClient.getJobStatus(jobGraph.getJobID()).get() == JobStatus.CANCELED, - Deadline.fromNow(Duration.ofMillis(TIMEOUT)), 1000); // Start a new job with savepoint 2 diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java index 5310a8b907efd..eddefe81a927b 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java @@ -118,6 +118,10 @@ TestingFlinkKubeClient.Builder createFlinkKubeClientBuilder() { return kubernetesTestFixture.createFlinkKubeClientBuilder(); } + String getClusterId() { + return CLUSTER_ID; + } + KubernetesConfigMap getLeaderConfigMap() { return kubernetesTestFixture.getLeaderConfigMap(); } @@ -125,7 +129,7 @@ KubernetesConfigMap getLeaderConfigMap() { // Use the leader callback to manually grant leadership void leaderCallbackGrantLeadership() throws Exception { kubernetesTestFixture.leaderCallbackGrantLeadership(); - electionEventHandler.waitForLeader(TIMEOUT); + electionEventHandler.waitForLeader(); } FlinkKubeClient.WatchCallbackHandler diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionAndRetrievalITCase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionAndRetrievalITCase.java index 6717a3a1c619e..694e8a83c0074 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionAndRetrievalITCase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionAndRetrievalITCase.java @@ -56,8 +56,6 @@ public class KubernetesLeaderElectionAndRetrievalITCase extends TestLogger { "akka.tcp://flink@172.20.1.21:6123/user/rpc/dispatcher"; @ClassRule public static KubernetesResource kubernetesResource = new KubernetesResource(); - private static final long TIMEOUT = 120L * 1000L; - @Test public void testLeaderElectionAndRetrieval() throws Exception { final String configMapName = LEADER_CONFIGMAP_NAME + System.currentTimeMillis(); @@ -101,14 +99,14 @@ public void testLeaderElectionAndRetrieval() throws Exception { KubernetesUtils::getLeaderInformationFromConfigMap, retrievalEventHandler::handleError); - electionEventHandler.waitForLeader(TIMEOUT); + electionEventHandler.waitForLeader(); // Check the new leader is confirmed final LeaderInformation confirmedLeaderInformation = electionEventHandler.getConfirmedLeaderInformation(); assertThat(confirmedLeaderInformation.getLeaderAddress(), is(LEADER_ADDRESS)); // Check the leader retrieval driver should be notified the leader address - retrievalEventHandler.waitForNewLeader(TIMEOUT); + retrievalEventHandler.waitForNewLeader(); assertThat( retrievalEventHandler.getLeaderSessionID(), is(confirmedLeaderInformation.getLeaderSessionID())); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriverTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriverTest.java index fa5cc17150844..6072051ea58e0 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriverTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriverTest.java @@ -70,7 +70,7 @@ public void testNotLeader() throws Exception { // Revoke leadership getLeaderCallback().notLeader(); - electionEventHandler.waitForRevokeLeader(TIMEOUT); + electionEventHandler.waitForRevokeLeader(); assertThat(electionEventHandler.isLeader(), is(false)); assertThat( electionEventHandler.getConfirmedLeaderInformation(), @@ -94,7 +94,7 @@ public void testHasLeadershipWhenConfigMapNotExist() throws Exception { runTest( () -> { leaderElectionDriver.hasLeadership(); - electionEventHandler.waitForError(TIMEOUT); + electionEventHandler.waitForError(); final String errorMsg = "ConfigMap " + LEADER_CONFIGMAP_NAME + " does not exist."; assertThat(electionEventHandler.getError(), is(notNullValue())); @@ -137,7 +137,7 @@ public void testWriteLeaderInformationWhenConfigMapNotExist() throws Exception { () -> { leaderElectionDriver.writeLeaderInformation( LeaderInformation.known(UUID.randomUUID(), LEADER_ADDRESS)); - electionEventHandler.waitForError(TIMEOUT); + electionEventHandler.waitForError(); final String errorMsg = "Could not write leader information since ConfigMap " @@ -205,7 +205,7 @@ public void testLeaderConfigMapDeletedExternally() throws Exception { callbackHandler.onDeleted( Collections.singletonList(getLeaderConfigMap())); - electionEventHandler.waitForError(TIMEOUT); + electionEventHandler.waitForError(); final String errorMsg = "ConfigMap " + LEADER_CONFIGMAP_NAME + " is deleted externally"; assertThat(electionEventHandler.getError(), is(notNullValue())); @@ -230,7 +230,7 @@ public void testErrorForwarding() throws Exception { callbackHandler.onError( Collections.singletonList(getLeaderConfigMap())); - electionEventHandler.waitForError(TIMEOUT); + electionEventHandler.waitForError(); final String errorMsg = "Error while watching the ConfigMap " + LEADER_CONFIGMAP_NAME; assertThat(electionEventHandler.getError(), is(notNullValue())); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderRetrievalDriverTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderRetrievalDriverTest.java index acd261307f35b..3193f9f15494c 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderRetrievalDriverTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderRetrievalDriverTest.java @@ -48,7 +48,7 @@ public void testErrorForwarding() throws Exception { Collections.singletonList(getLeaderConfigMap())); final String errMsg = "Error while watching the ConfigMap " + LEADER_CONFIGMAP_NAME; - retrievalEventHandler.waitForError(TIMEOUT); + retrievalEventHandler.waitForError(); assertThat( retrievalEventHandler.getError(), FlinkMatchers.containsMessage(errMsg)); @@ -76,8 +76,7 @@ public void testKubernetesLeaderRetrievalOnModified() throws Exception { callbackHandler.onModified( Collections.singletonList(getLeaderConfigMap())); - assertThat( - retrievalEventHandler.waitForNewLeader(TIMEOUT), is(newLeader)); + assertThat(retrievalEventHandler.waitForNewLeader(), is(newLeader)); }); } }; @@ -98,7 +97,7 @@ public void testKubernetesLeaderRetrievalOnModifiedWithEmpty() throws Exception getLeaderConfigMap().getData().clear(); callbackHandler.onModified( Collections.singletonList(getLeaderConfigMap())); - retrievalEventHandler.waitForEmptyLeaderInformation(TIMEOUT); + retrievalEventHandler.waitForEmptyLeaderInformation(); assertThat(retrievalEventHandler.getAddress(), is(nullValue())); }); } diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriverTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriverTest.java index c4c0a180cd33f..37d64771e8df3 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriverTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriverTest.java @@ -118,7 +118,7 @@ public void testPublishLeaderInformation() throws Exception { notifyLeaderRetrievalWatchOnModifiedConfigMap(); - leaderRetrievalListener.waitForNewLeader(10_000L); + leaderRetrievalListener.waitForNewLeader(); assertThat(leaderRetrievalListener.getLeader()) .isEqualTo(leaderInformation); }); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStoreITCase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStoreITCase.java index f4467c47f7d93..5c78168953e4b 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStoreITCase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStoreITCase.java @@ -51,8 +51,6 @@ public class KubernetesStateHandleStoreITCase extends TestLogger { private final FlinkKubeClientFactory kubeClientFactory = new FlinkKubeClientFactory(); - private static final long TIMEOUT = 120L * 1000L; - private static final String KEY = "state-handle-test"; @Test @@ -94,8 +92,7 @@ public void testMultipleKubernetesStateHandleStores() throws Exception { } // Wait for the leader - final String lockIdentity = - TestingLeaderCallbackHandler.waitUntilNewLeaderAppears(TIMEOUT); + final String lockIdentity = TestingLeaderCallbackHandler.waitUntilNewLeaderAppears(); Long expectedState = null; for (int i = 0; i < leaderNum; i++) { diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStoreTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStoreTest.java index 9896d49512d2d..c2b46c8ed808e 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStoreTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStoreTest.java @@ -445,7 +445,7 @@ public void testReplaceWithNoLeadershipAndDiscardState() throws Exception { store.addAndLock(key, state); // Lost leadership getLeaderCallback().notLeader(); - electionEventHandler.waitForRevokeLeader(TIMEOUT); + electionEventHandler.waitForRevokeLeader(); getLeaderConfigMap() .getAnnotations() .remove(KubernetesLeaderElector.LEADER_ANNOTATION_KEY); @@ -803,6 +803,33 @@ public void testRemove() throws Exception { }; } + @Test + public void testRemoveOfNonExistingState() throws Exception { + new Context() { + { + runTest( + () -> { + leaderCallbackGrantLeadership(); + + final KubernetesStateHandleStore< + TestingLongStateHandleHelper.LongStateHandle> + store = + new KubernetesStateHandleStore<>( + flinkKubeClient, + LEADER_CONFIGMAP_NAME, + longStateStorage, + filter, + LOCK_IDENTITY); + assertThat(store.getAllAndLock().size(), is(0)); + assertThat(store.releaseAndTryRemove(key), is(true)); + assertThat(store.getAllAndLock().size(), is(0)); + + assertThat(TestingLongStateHandleHelper.getGlobalDiscardCount(), is(0)); + }); + } + }; + } + @Test public void testRemoveFailedShouldNotDiscardState() throws Exception { new Context() { @@ -824,7 +851,7 @@ public void testRemoveFailedShouldNotDiscardState() throws Exception { store.addAndLock(key, state); // Lost leadership getLeaderCallback().notLeader(); - electionEventHandler.waitForRevokeLeader(TIMEOUT); + electionEventHandler.waitForRevokeLeader(); getLeaderConfigMap() .getAnnotations() .remove(KubernetesLeaderElector.LEADER_ANNOTATION_KEY); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElectorITCase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElectorITCase.java index 940d8dcfe0235..bce3069422626 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElectorITCase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElectorITCase.java @@ -42,8 +42,6 @@ public class KubernetesLeaderElectorITCase extends TestLogger { @ClassRule public static KubernetesResource kubernetesResource = new KubernetesResource(); - private static final long TIMEOUT = 120L * 1000L; - private final FlinkKubeClientFactory kubeClientFactory = new FlinkKubeClientFactory(); private static final String LEADER_CONFIGMAP_NAME_PREFIX = "leader-test-cluster"; @@ -81,17 +79,17 @@ public void testMultipleKubernetesLeaderElectors() throws Exception { // Wait for the first leader final String firstLockIdentity = - TestingLeaderCallbackHandler.waitUntilNewLeaderAppears(TIMEOUT); + TestingLeaderCallbackHandler.waitUntilNewLeaderAppears(); for (int i = 0; i < leaderNum; i++) { if (leaderCallbackHandlers[i].getLockIdentity().equals(firstLockIdentity)) { // Check the callback isLeader is called. - leaderCallbackHandlers[i].waitForNewLeader(TIMEOUT); + leaderCallbackHandlers[i].waitForNewLeader(); assertThat(leaderCallbackHandlers[i].hasLeadership(), is(true)); // Current leader died leaderElectors[i].stop(); // Check the callback notLeader is called. - leaderCallbackHandlers[i].waitForRevokeLeader(TIMEOUT); + leaderCallbackHandlers[i].waitForRevokeLeader(); assertThat(leaderCallbackHandlers[i].hasLeadership(), is(false)); } else { assertThat(leaderCallbackHandlers[i].hasLeadership(), is(false)); @@ -100,7 +98,7 @@ public void testMultipleKubernetesLeaderElectors() throws Exception { // Another leader should be elected successfully and update the lock identity final String anotherLockIdentity = - TestingLeaderCallbackHandler.waitUntilNewLeaderAppears(TIMEOUT); + TestingLeaderCallbackHandler.waitUntilNewLeaderAppears(); assertThat(anotherLockIdentity, is(not(firstLockIdentity))); } finally { // Cleanup the resources diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/TestingLeaderCallbackHandler.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/TestingLeaderCallbackHandler.java index edc1e70e483fb..bdfd86ff534d2 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/TestingLeaderCallbackHandler.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/TestingLeaderCallbackHandler.java @@ -18,14 +18,10 @@ package org.apache.flink.kubernetes.kubeclient.resources; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.runtime.testutils.CommonTestUtils; -import java.time.Duration; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; /** Testing implementation for {@link KubernetesLeaderElector.LeaderCallbackHandler}. */ public class TestingLeaderCallbackHandler extends KubernetesLeaderElector.LeaderCallbackHandler { @@ -63,38 +59,23 @@ public boolean hasLeadership() { return isLeader; } - public static String waitUntilNewLeaderAppears(long timeout) throws Exception { - final AtomicReference leaderRef = new AtomicReference<>(); - CommonTestUtils.waitUntilCondition( - () -> { - final String lockIdentity = sharedQueue.poll(timeout, TimeUnit.MILLISECONDS); - leaderRef.set(lockIdentity); - return lockIdentity != null; - }, - Deadline.fromNow(Duration.ofMillis(timeout)), - "No leader is elected with " + timeout + "ms"); - return leaderRef.get(); + public static String waitUntilNewLeaderAppears() throws Exception { + return sharedQueue.take(); } - public void waitForNewLeader(long timeout) throws Exception { - final String errorMsg = - "No leader with " + lockIdentity + " is elected within " + timeout + "ms"; - poll(leaderQueue, timeout, errorMsg); + public void waitForNewLeader() throws Exception { + poll(leaderQueue); } - public void waitForRevokeLeader(long timeout) throws Exception { - final String errorMsg = - "No leader with " + lockIdentity + " is revoke within " + timeout + "ms"; - poll(revokeQueue, timeout, errorMsg); + public void waitForRevokeLeader() throws Exception { + poll(revokeQueue); } - private void poll(BlockingQueue queue, long timeout, String errorMsg) throws Exception { + private void poll(BlockingQueue queue) throws Exception { CommonTestUtils.waitUntilCondition( () -> { - final String lockIdentity = queue.poll(timeout, TimeUnit.MILLISECONDS); + final String lockIdentity = queue.take(); return this.lockIdentity.equals(lockIdentity); - }, - Deadline.fromNow(Duration.ofMillis(timeout)), - errorMsg); + }); } } diff --git a/flink-libraries/flink-cep-scala/pom.xml b/flink-libraries/flink-cep-scala/pom.xml index ac0dae0f6d933..1258dd8c9a925 100644 --- a/flink-libraries/flink-cep-scala/pom.xml +++ b/flink-libraries/flink-cep-scala/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-libraries - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-libraries/flink-cep/pom.xml b/flink-libraries/flink-cep/pom.xml index 5dcdaab52afac..9613479149359 100644 --- a/flink-libraries/flink-cep/pom.xml +++ b/flink-libraries/flink-cep/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-libraries - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-libraries/flink-gelly-examples/pom.xml b/flink-libraries/flink-gelly-examples/pom.xml index d53855f70935e..193bef3135698 100644 --- a/flink-libraries/flink-gelly-examples/pom.xml +++ b/flink-libraries/flink-gelly-examples/pom.xml @@ -23,7 +23,7 @@ org.apache.flink flink-libraries - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-libraries/flink-gelly-scala/pom.xml b/flink-libraries/flink-gelly-scala/pom.xml index b99752fd686c0..9506fd3ef826f 100644 --- a/flink-libraries/flink-gelly-scala/pom.xml +++ b/flink-libraries/flink-gelly-scala/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-libraries - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-libraries/flink-gelly/pom.xml b/flink-libraries/flink-gelly/pom.xml index c1d8248db3594..f71ad5968a697 100644 --- a/flink-libraries/flink-gelly/pom.xml +++ b/flink-libraries/flink-gelly/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-libraries - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-libraries/flink-state-processing-api/pom.xml b/flink-libraries/flink-state-processing-api/pom.xml index 3d13cb304c4de..c131b1622c6f8 100644 --- a/flink-libraries/flink-state-processing-api/pom.xml +++ b/flink-libraries/flink-state-processing-api/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-libraries - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-libraries/pom.xml b/flink-libraries/pom.xml index e746ab834a050..58c93b30db2a7 100644 --- a/flink-libraries/pom.xml +++ b/flink-libraries/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-metrics/flink-metrics-core/pom.xml b/flink-metrics/flink-metrics-core/pom.xml index 8895ef3629092..ba2cbf0382db5 100644 --- a/flink-metrics/flink-metrics-core/pom.xml +++ b/flink-metrics/flink-metrics-core/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-metrics - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-metrics/flink-metrics-datadog/pom.xml b/flink-metrics/flink-metrics-datadog/pom.xml index 0707ff0527a2f..ea0d877cc79ea 100644 --- a/flink-metrics/flink-metrics-datadog/pom.xml +++ b/flink-metrics/flink-metrics-datadog/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-metrics - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-metrics/flink-metrics-dropwizard/pom.xml b/flink-metrics/flink-metrics-dropwizard/pom.xml index 4009c866704ce..85a5033f743c2 100644 --- a/flink-metrics/flink-metrics-dropwizard/pom.xml +++ b/flink-metrics/flink-metrics-dropwizard/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-metrics - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-metrics/flink-metrics-graphite/pom.xml b/flink-metrics/flink-metrics-graphite/pom.xml index 509b1b7efff8f..f6b4cad50b866 100644 --- a/flink-metrics/flink-metrics-graphite/pom.xml +++ b/flink-metrics/flink-metrics-graphite/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-metrics - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-metrics/flink-metrics-influxdb/pom.xml b/flink-metrics/flink-metrics-influxdb/pom.xml index 27d5a6e9d2e89..790e3b2a773de 100644 --- a/flink-metrics/flink-metrics-influxdb/pom.xml +++ b/flink-metrics/flink-metrics-influxdb/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-metrics - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-metrics/flink-metrics-jmx/pom.xml b/flink-metrics/flink-metrics-jmx/pom.xml index af846b70e53c1..1734e440869d6 100644 --- a/flink-metrics/flink-metrics-jmx/pom.xml +++ b/flink-metrics/flink-metrics-jmx/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-metrics - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-metrics/flink-metrics-prometheus/pom.xml b/flink-metrics/flink-metrics-prometheus/pom.xml index 7f2b12dabe597..d14bd3adf2b1a 100644 --- a/flink-metrics/flink-metrics-prometheus/pom.xml +++ b/flink-metrics/flink-metrics-prometheus/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-metrics - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-metrics/flink-metrics-slf4j/pom.xml b/flink-metrics/flink-metrics-slf4j/pom.xml index 53654fdc83c43..0731dae652e75 100644 --- a/flink-metrics/flink-metrics-slf4j/pom.xml +++ b/flink-metrics/flink-metrics-slf4j/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-metrics - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-metrics/flink-metrics-statsd/pom.xml b/flink-metrics/flink-metrics-statsd/pom.xml index 399e22cdbf2ed..d67499f01a2d6 100644 --- a/flink-metrics/flink-metrics-statsd/pom.xml +++ b/flink-metrics/flink-metrics-statsd/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-metrics - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-metrics/pom.xml b/flink-metrics/pom.xml index 9d5e69c94a442..3469289c6a671 100644 --- a/flink-metrics/pom.xml +++ b/flink-metrics/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-optimizer/pom.xml b/flink-optimizer/pom.xml index 18cf197c5c1f5..6b2f7150d2951 100644 --- a/flink-optimizer/pom.xml +++ b/flink-optimizer/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-python/dev/dev-requirements.txt b/flink-python/dev/dev-requirements.txt index bd75137d024bf..b061a67a20d28 100755 --- a/flink-python/dev/dev-requirements.txt +++ b/flink-python/dev/dev-requirements.txt @@ -27,4 +27,4 @@ numpy>=1.14.3,<1.20 fastavro>=0.21.4,<0.24 grpcio>=1.29.0,<2 grpcio-tools>=1.3.5,<=1.14.2 -pemja==0.1.2; python_version >= '3.7' +pemja==0.1.4; python_version >= '3.7' diff --git a/flink-python/dev/lint-python.sh b/flink-python/dev/lint-python.sh index c5dc80dbd4f19..789720208ec72 100755 --- a/flink-python/dev/lint-python.sh +++ b/flink-python/dev/lint-python.sh @@ -322,7 +322,7 @@ function install_sphinx() { fi fi - $CURRENT_DIR/install_command.sh -q Sphinx==2.4.4 Docutils==0.17.1 2>&1 >/dev/null + $CURRENT_DIR/install_command.sh -q Sphinx==2.4.4 Docutils==0.17.1 "Jinja2<3.1.0" 2>&1 >/dev/null if [ $? -ne 0 ]; then echo "pip install sphinx failed \ please try to exec the script again.\ diff --git a/flink-python/pom.xml b/flink-python/pom.xml index ad39eea7b1681..77e2a87afff94 100644 --- a/flink-python/pom.xml +++ b/flink-python/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. @@ -107,7 +107,7 @@ under the License. com.alibaba pemja - 0.1.2 + 0.1.4 diff --git a/tools/ci/build_docs.sh b/flink-python/pyflink/common/constants.py old mode 100755 new mode 100644 similarity index 66% rename from tools/ci/build_docs.sh rename to flink-python/pyflink/common/constants.py index 165d99f0e9bd1..3fe6e0e1a0d84 --- a/tools/ci/build_docs.sh +++ b/flink-python/pyflink/common/constants.py @@ -1,4 +1,3 @@ -#!/usr/bin/env bash ################################################################################ # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -16,23 +15,12 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ +""" +A constant holding the maximum value a long can have, 2^63 – 1. +""" +MAX_LONG_VALUE = 0x7fffffffffffffff -HUGO_REPO=https://github.com/gohugoio/hugo/releases/download/v0.80.0/hugo_extended_0.80.0_Linux-64bit.tar.gz -HUGO_ARTIFACT=hugo_extended_0.80.0_Linux-64bit.tar.gz - -if ! curl --fail -OL $HUGO_REPO ; then - echo "Failed to download Hugo binary" - exit 1 -fi - -tar -zxvf $HUGO_ARTIFACT - -git submodule update --init --recursive -# generate docs into docs/target -./hugo -v --source docs --destination target - -if [ $? -ne 0 ]; then - echo "Error building the docs" - exit 1 -fi - +""" +A constant holding the minimum value a long can have, -2^63 +""" +MIN_LONG_VALUE = - MAX_LONG_VALUE - 1 diff --git a/flink-python/pyflink/datastream/connectors/pulsar.py b/flink-python/pyflink/datastream/connectors/pulsar.py new file mode 100644 index 0000000000000..4f4c60405e274 --- /dev/null +++ b/flink-python/pyflink/datastream/connectors/pulsar.py @@ -0,0 +1,774 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ +import warnings +from enum import Enum +from typing import Dict, Union, List + +from pyflink.common import DeserializationSchema, TypeInformation, ExecutionConfig, \ + ConfigOptions, Duration, SerializationSchema, ConfigOption +from pyflink.datastream.connectors import Source, Sink, DeliveryGuarantee +from pyflink.java_gateway import get_gateway +from pyflink.util.java_utils import load_java_class + + +# ---- PulsarSource ---- + + +class PulsarDeserializationSchema(object): + """ + A schema bridge for deserializing the pulsar's Message into a flink managed instance. We + support both the pulsar's self managed schema and flink managed schema. + """ + + def __init__(self, _j_pulsar_deserialization_schema): + self._j_pulsar_deserialization_schema = _j_pulsar_deserialization_schema + + @staticmethod + def flink_schema(deserialization_schema: DeserializationSchema) \ + -> 'PulsarDeserializationSchema': + """ + Create a PulsarDeserializationSchema by using the flink's DeserializationSchema. It would + consume the pulsar message as byte array and decode the message by using flink's logic. + """ + JPulsarDeserializationSchema = get_gateway().jvm.org.apache.flink \ + .connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema + _j_pulsar_deserialization_schema = JPulsarDeserializationSchema.flinkSchema( + deserialization_schema._j_deserialization_schema) + return PulsarDeserializationSchema(_j_pulsar_deserialization_schema) + + @staticmethod + def flink_type_info(type_information: TypeInformation, + execution_config: ExecutionConfig = None) -> 'PulsarDeserializationSchema': + """ + Create a PulsarDeserializationSchema by using the given TypeInformation. This method is + only used for treating message that was written into pulsar by TypeInformation. + """ + JPulsarDeserializationSchema = get_gateway().jvm.org.apache.flink \ + .connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema + JExecutionConfig = get_gateway().jvm.org.apache.flink.api.common.ExecutionConfig + _j_execution_config = execution_config._j_execution_config \ + if execution_config is not None else JExecutionConfig() + _j_pulsar_deserialization_schema = JPulsarDeserializationSchema.flinkTypeInfo( + type_information.get_java_type_info(), _j_execution_config) + return PulsarDeserializationSchema(_j_pulsar_deserialization_schema) + + +class SubscriptionType(Enum): + """ + Types of subscription supported by Pulsar. + + :data: `Exclusive`: + + There can be only 1 consumer on the same topic with the same subscription name. + + :data: `Shared`: + + Multiple consumer will be able to use the same subscription name and the messages will be + dispatched according to a round-robin rotation between the connected consumers. In this mode, + the consumption order is not guaranteed. + + :data: `Failover`: + + Multiple consumer will be able to use the same subscription name but only 1 consumer will + receive the messages. If that consumer disconnects, one of the other connected consumers will + start receiving messages. In failover mode, the consumption ordering is guaranteed. In case of + partitioned topics, the ordering is guaranteed on a per-partition basis. The partitions + assignments will be split across the available consumers. On each partition, at most one + consumer will be active at a given point in time. + + :data: `Key_Shared`: + + Multiple consumer will be able to use the same subscription and all messages with the same key + will be dispatched to only one consumer. Use ordering_key to overwrite the message key for + message ordering. + """ + + Exclusive = 0, + Shared = 1, + Failover = 2, + Key_Shared = 3 + + def _to_j_subscription_type(self): + JSubscriptionType = get_gateway().jvm.org.apache.pulsar.client.api.SubscriptionType + return getattr(JSubscriptionType, self.name) + + +class StartCursor(object): + """ + A factory class for users to specify the start position of a pulsar subscription. + Since it would be serialized into split. + The implementation for this interface should be well considered. + I don't recommend adding extra internal state for this implementation. + + This class would be used only for SubscriptionType.Exclusive and SubscriptionType.Failover. + """ + + def __init__(self, _j_start_cursor): + self._j_start_cursor = _j_start_cursor + + @staticmethod + def default_start_cursor() -> 'StartCursor': + return StartCursor.earliest() + + @staticmethod + def earliest() -> 'StartCursor': + JStartCursor = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor + return StartCursor(JStartCursor.earliest()) + + @staticmethod + def latest() -> 'StartCursor': + JStartCursor = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor + return StartCursor(JStartCursor.latest()) + + @staticmethod + def from_publish_time(timestamp: int) -> 'StartCursor': + JStartCursor = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor + return StartCursor(JStartCursor.fromPublishTime(timestamp)) + + @staticmethod + def from_message_id(message_id: bytes, inclusive: bool = True) -> 'StartCursor': + """ + Find the available message id and start consuming from it. User could call pulsar Python + library serialize method to cover messageId bytes. + + Example: + :: + + >>> from pulsar import MessageId + >>> message_id_bytes = MessageId().serialize() + >>> start_cursor = StartCursor.from_message_id(message_id_bytes) + """ + JStartCursor = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor + j_message_id = get_gateway().jvm.org.apache.pulsar.client.api.MessageId \ + .fromByteArray(message_id) + return StartCursor(JStartCursor.fromMessageId(j_message_id, inclusive)) + + +class StopCursor(object): + """ + A factory class for users to specify the stop position of a pulsar subscription. Since it would + be serialized into split. The implementation for this interface should be well considered. I + don't recommend adding extra internal state for this implementation. + """ + + def __init__(self, _j_stop_cursor): + self._j_stop_cursor = _j_stop_cursor + + @staticmethod + def default_stop_cursor() -> 'StopCursor': + return StopCursor.never() + + @staticmethod + def never() -> 'StopCursor': + JStopCursor = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor + return StopCursor(JStopCursor.never()) + + @staticmethod + def latest() -> 'StopCursor': + JStopCursor = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor + return StopCursor(JStopCursor.latest()) + + @staticmethod + def at_event_time(timestamp: int) -> 'StopCursor': + warnings.warn( + "at_event_time is deprecated. Use at_publish_time instead.", DeprecationWarning) + JStopCursor = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor + return StopCursor(JStopCursor.atEventTime(timestamp)) + + @staticmethod + def at_publish_time(timestamp: int) -> 'StopCursor': + """ + Stop when message publishTime is greater than the specified timestamp. + """ + JStopCursor = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor + return StopCursor(JStopCursor.atPublishTime(timestamp)) + + @staticmethod + def at_message_id(message_id: bytes) -> 'StopCursor': + """ + Stop when the messageId is equal or greater than the specified messageId. Message that is + equal to the specified messageId will not be consumed. User could call pulsar Python + library serialize method to cover messageId bytes. + + Example: + :: + + >>> from pulsar import MessageId + >>> message_id_bytes = MessageId().serialize() + >>> stop_cursor = StopCursor.at_message_id(message_id_bytes) + """ + JStopCursor = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor + j_message_id = get_gateway().jvm.org.apache.pulsar.client.api.MessageId \ + .fromByteArray(message_id) + return StopCursor(JStopCursor.atMessageId(j_message_id)) + + @staticmethod + def after_message_id(message_id: bytes) -> 'StopCursor': + """ + Stop when the messageId is greater than the specified messageId. Message that is equal to + the specified messageId will be consumed. User could call pulsar Python library serialize + method to cover messageId bytes. + + Example: + :: + + >>> from pulsar import MessageId + >>> message_id_bytes = MessageId().serialize() + >>> stop_cursor = StopCursor.after_message_id(message_id_bytes) + """ + JStopCursor = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor + j_message_id = get_gateway().jvm.org.apache.pulsar.client.api.MessageId \ + .fromByteArray(message_id) + return StopCursor(JStopCursor.afterMessageId(j_message_id)) + + +class PulsarSource(Source): + """ + The Source implementation of Pulsar. Please use a PulsarSourceBuilder to construct a + PulsarSource. The following example shows how to create a PulsarSource emitting records of + String type. + + Example: + :: + + >>> source = PulsarSource() \\ + ... .builder() \\ + ... .set_topics([TOPIC1, TOPIC2]) \\ + ... .set_service_url(get_service_url()) \\ + ... .set_admin_url(get_admin_url()) \\ + ... .set_subscription_name("test") \\ + ... .set_deserialization_schema( + ... PulsarDeserializationSchema.flink_schema(SimpleStringSchema())) \\ + ... .set_bounded_stop_cursor(StopCursor.default_stop_cursor()) \\ + ... .build() + + See PulsarSourceBuilder for more details. + """ + + def __init__(self, j_pulsar_source): + super(PulsarSource, self).__init__(source=j_pulsar_source) + + @staticmethod + def builder() -> 'PulsarSourceBuilder': + """ + Get a PulsarSourceBuilder to builder a PulsarSource. + """ + return PulsarSourceBuilder() + + +class PulsarSourceBuilder(object): + """ + The builder class for PulsarSource to make it easier for the users to construct a PulsarSource. + + The following example shows the minimum setup to create a PulsarSource that reads the String + values from a Pulsar topic. + + Example: + :: + + >>> source = PulsarSource() \\ + ... .builder() \\ + ... .set_service_url(PULSAR_BROKER_URL) \\ + ... .set_admin_url(PULSAR_BROKER_HTTP_URL) \\ + ... .set_subscription_name("flink-source-1") \\ + ... .set_topics([TOPIC1, TOPIC2]) \\ + ... .set_deserialization_schema( + ... PulsarDeserializationSchema.flink_schema(SimpleStringSchema())) \\ + ... .build() + + The service url, admin url, subscription name, topics to consume, and the record deserializer + are required fields that must be set. + + To specify the starting position of PulsarSource, one can call set_start_cursor(StartCursor). + + By default the PulsarSource runs in an Boundedness.CONTINUOUS_UNBOUNDED mode and never stop + until the Flink job is canceled or fails. To let the PulsarSource run in + Boundedness.CONTINUOUS_UNBOUNDED but stops at some given offsets, one can call + set_unbounded_stop_cursor(StopCursor). + + For example the following PulsarSource stops after it consumes up to a event time when the + Flink started. + + Example: + :: + + >>> source = PulsarSource() \\ + ... .builder() \\ + ... .set_service_url(PULSAR_BROKER_URL) \\ + ... .set_admin_url(PULSAR_BROKER_HTTP_URL) \\ + ... .set_subscription_name("flink-source-1") \\ + ... .set_topics([TOPIC1, TOPIC2]) \\ + ... .set_deserialization_schema( + ... PulsarDeserializationSchema.flink_schema(SimpleStringSchema())) \\ + ... .set_bounded_stop_cursor(StopCursor.at_publish_time(int(time.time() * 1000))) + ... .build() + """ + + def __init__(self): + JPulsarSource = \ + get_gateway().jvm.org.apache.flink.connector.pulsar.source.PulsarSource + self._j_pulsar_source_builder = JPulsarSource.builder() + + def set_admin_url(self, admin_url: str) -> 'PulsarSourceBuilder': + """ + Sets the admin endpoint for the PulsarAdmin of the PulsarSource. + """ + self._j_pulsar_source_builder.setAdminUrl(admin_url) + return self + + def set_service_url(self, service_url: str) -> 'PulsarSourceBuilder': + """ + Sets the server's link for the PulsarConsumer of the PulsarSource. + """ + self._j_pulsar_source_builder.setServiceUrl(service_url) + return self + + def set_subscription_name(self, subscription_name: str) -> 'PulsarSourceBuilder': + """ + Sets the name for this pulsar subscription. + """ + self._j_pulsar_source_builder.setSubscriptionName(subscription_name) + return self + + def set_subscription_type(self, subscription_type: SubscriptionType) -> 'PulsarSourceBuilder': + """ + SubscriptionType is the consuming behavior for pulsar, we would generator different split + by the given subscription type. Please take some time to consider which subscription type + matches your application best. Default is SubscriptionType.Shared. + """ + self._j_pulsar_source_builder.setSubscriptionType( + subscription_type._to_j_subscription_type()) + return self + + def set_topics(self, topics: Union[str, List[str]]) -> 'PulsarSourceBuilder': + """ + Set a pulsar topic list for flink source. Some topic may not exist currently, consuming this + non-existed topic wouldn't throw any exception. But the best solution is just consuming by + using a topic regex. You can set topics once either with setTopics or setTopicPattern in + this builder. + """ + if not isinstance(topics, list): + topics = [topics] + self._j_pulsar_source_builder.setTopics(topics) + return self + + def set_topics_pattern(self, topics_pattern: str) -> 'PulsarSourceBuilder': + """ + Set a topic pattern to consume from the java regex str. You can set topics once either with + set_topics or set_topic_pattern in this builder. + """ + warnings.warn("set_topics_pattern is deprecated. Use set_topic_pattern instead.", + DeprecationWarning, stacklevel=2) + self._j_pulsar_source_builder.setTopicPattern(topics_pattern) + return self + + def set_topic_pattern(self, topic_pattern: str) -> 'PulsarSourceBuilder': + """ + Set a topic pattern to consume from the java regex str. You can set topics once either with + set_topics or set_topic_pattern in this builder. + """ + self._j_pulsar_source_builder.setTopicPattern(topic_pattern) + return self + + def set_start_cursor(self, start_cursor: StartCursor) -> 'PulsarSourceBuilder': + """ + Specify from which offsets the PulsarSource should start consume from by providing an + StartCursor. + """ + self._j_pulsar_source_builder.setStartCursor(start_cursor._j_start_cursor) + return self + + def set_unbounded_stop_cursor(self, stop_cursor: StopCursor) -> 'PulsarSourceBuilder': + """ + By default the PulsarSource is set to run in Boundedness.CONTINUOUS_UNBOUNDED manner and + thus never stops until the Flink job fails or is canceled. To let the PulsarSource run as a + streaming source but still stops at some point, one can set an StopCursor to specify the + stopping offsets for each partition. When all the partitions have reached their stopping + offsets, the PulsarSource will then exit. + + This method is different from set_bounded_stop_cursor(StopCursor) that after setting the + stopping offsets with this method, PulsarSource.getBoundedness() will still return + Boundedness.CONTINUOUS_UNBOUNDED even though it will stop at the stopping offsets specified + by the stopping offsets StopCursor. + """ + self._j_pulsar_source_builder.setUnboundedStopCursor(stop_cursor._j_stop_cursor) + return self + + def set_bounded_stop_cursor(self, stop_cursor: StopCursor) -> 'PulsarSourceBuilder': + """ + By default the PulsarSource is set to run in Boundedness.CONTINUOUS_UNBOUNDED manner and + thus never stops until the Flink job fails or is canceled. To let the PulsarSource run in + Boundedness.BOUNDED manner and stops at some point, one can set an StopCursor to specify + the stopping offsets for each partition. When all the partitions have reached their stopping + offsets, the PulsarSource will then exit. + + This method is different from set_unbounded_stop_cursor(StopCursor) that after setting the + stopping offsets with this method, PulsarSource.getBoundedness() will return + Boundedness.BOUNDED instead of Boundedness.CONTINUOUS_UNBOUNDED. + """ + self._j_pulsar_source_builder.setBoundedStopCursor(stop_cursor._j_stop_cursor) + return self + + def set_deserialization_schema(self, + pulsar_deserialization_schema: PulsarDeserializationSchema) \ + -> 'PulsarSourceBuilder': + """ + DeserializationSchema is required for getting the Schema for deserialize message from + pulsar and getting the TypeInformation for message serialization in flink. + + We have defined a set of implementations, using PulsarDeserializationSchema#flink_type_info + or PulsarDeserializationSchema#flink_schema for creating the desired schema. + """ + self._j_pulsar_source_builder.setDeserializationSchema( + pulsar_deserialization_schema._j_pulsar_deserialization_schema) + return self + + def set_config(self, key: Union[str, ConfigOption], value) -> 'PulsarSourceBuilder': + """ + Set arbitrary properties for the PulsarSource and PulsarConsumer. The valid keys can be + found in PulsarSourceOptions and PulsarOptions. + + Make sure the option could be set only once or with same value. + """ + if isinstance(key, ConfigOption): + warnings.warn("set_config(key: ConfigOption, value) is deprecated. " + "Use set_config(key: str, value) instead.", + DeprecationWarning, stacklevel=2) + j_config_option = key._j_config_option + else: + j_config_option = \ + ConfigOptions.key(key).string_type().no_default_value()._j_config_option + self._j_pulsar_source_builder.setConfig(j_config_option, value) + return self + + def set_config_with_dict(self, config: Dict) -> 'PulsarSourceBuilder': + """ + Set arbitrary properties for the PulsarSource and PulsarConsumer. The valid keys can be + found in PulsarSourceOptions and PulsarOptions. + """ + warnings.warn("set_config_with_dict is deprecated. Use set_properties instead.", + DeprecationWarning, stacklevel=2) + self.set_properties(config) + return self + + def set_properties(self, config: Dict) -> 'PulsarSourceBuilder': + """ + Set arbitrary properties for the PulsarSource and PulsarConsumer. The valid keys can be + found in PulsarSourceOptions and PulsarOptions. + """ + JConfiguration = get_gateway().jvm.org.apache.flink.configuration.Configuration + self._j_pulsar_source_builder.setConfig(JConfiguration.fromMap(config)) + return self + + def build(self) -> 'PulsarSource': + """ + Build the PulsarSource. + """ + return PulsarSource(self._j_pulsar_source_builder.build()) + + +# ---- PulsarSink ---- + + +class PulsarSerializationSchema(object): + """ + The serialization schema for how to serialize records into Pulsar. + """ + + def __init__(self, _j_pulsar_serialization_schema): + self._j_pulsar_serialization_schema = _j_pulsar_serialization_schema + + @staticmethod + def flink_schema(serialization_schema: SerializationSchema) \ + -> 'PulsarSerializationSchema': + """ + Create a PulsarSerializationSchema by using the flink's SerializationSchema. It would + serialize the message into byte array and send it to Pulsar with Schema#BYTES. + """ + JPulsarSerializationSchema = get_gateway().jvm.org.apache.flink \ + .connector.pulsar.sink.writer.serializer.PulsarSerializationSchema + _j_pulsar_serialization_schema = JPulsarSerializationSchema.flinkSchema( + serialization_schema._j_serialization_schema) + return PulsarSerializationSchema(_j_pulsar_serialization_schema) + + +class TopicRoutingMode(Enum): + """ + The routing policy for choosing the desired topic by the given message. + + :data: `ROUND_ROBIN`: + + The producer will publish messages across all partitions in a round-robin fashion to achieve + maximum throughput. Please note that round-robin is not done per individual message but + rather it's set to the same boundary of batching delay, to ensure batching is effective. + + :data: `MESSAGE_KEY_HASH`: + + If no key is provided, The partitioned producer will randomly pick one single topic partition + and publish all the messages into that partition. If a key is provided on the message, the + partitioned producer will hash the key and assign the message to a particular partition. + + :data: `CUSTOM`: + + Use custom topic router implementation that will be called to determine the partition for a + particular message. + """ + + ROUND_ROBIN = 0 + MESSAGE_KEY_HASH = 1 + CUSTOM = 2 + + def _to_j_topic_routing_mode(self): + JTopicRoutingMode = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode + return getattr(JTopicRoutingMode, self.name) + + +class MessageDelayer(object): + """ + A delayer for Pulsar broker passing the sent message to the downstream consumer. This is only + works in :data:`SubscriptionType.Shared` subscription. + + Read delayed message delivery + https://pulsar.apache.org/docs/en/next/concepts-messaging/#delayed-message-delivery for better + understanding this feature. + """ + def __init__(self, _j_message_delayer): + self._j_message_delayer = _j_message_delayer + + @staticmethod + def never() -> 'MessageDelayer': + """ + All the messages should be consumed immediately. + """ + JMessageDelayer = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.sink.writer.delayer.MessageDelayer + return MessageDelayer(JMessageDelayer.never()) + + @staticmethod + def fixed(duration: Duration) -> 'MessageDelayer': + """ + All the messages should be consumed in a fixed duration. + """ + JMessageDelayer = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.sink.writer.delayer.MessageDelayer + return MessageDelayer(JMessageDelayer.fixed(duration._j_duration)) + + +class PulsarSink(Sink): + """ + The Sink implementation of Pulsar. Please use a PulsarSinkBuilder to construct a + PulsarSink. The following example shows how to create a PulsarSink receiving records of + String type. + + Example: + :: + + >>> sink = PulsarSink.builder() \\ + ... .set_service_url(PULSAR_BROKER_URL) \\ + ... .set_admin_url(PULSAR_BROKER_HTTP_URL) \\ + ... .set_topics(topic) \\ + ... .set_serialization_schema( + ... PulsarSerializationSchema.flink_schema(SimpleStringSchema())) \\ + ... .build() + + The sink supports all delivery guarantees described by DeliveryGuarantee. + + DeliveryGuarantee#NONE does not provide any guarantees: messages may be lost in + case of issues on the Pulsar broker and messages may be duplicated in case of a Flink + failure. + + DeliveryGuarantee#AT_LEAST_ONCE the sink will wait for all outstanding records in + the Pulsar buffers to be acknowledged by the Pulsar producer on a checkpoint. No messages + will be lost in case of any issue with the Pulsar brokers but messages may be duplicated + when Flink restarts. + + DeliveryGuarantee#EXACTLY_ONCE: In this mode the PulsarSink will write all messages + in a Pulsar transaction that will be committed to Pulsar on a checkpoint. Thus, no + duplicates will be seen in case of a Flink restart. However, this delays record writing + effectively until a checkpoint is written, so adjust the checkpoint duration accordingly. + Additionally, it is highly recommended to tweak Pulsar transaction timeout (link) >> + maximum checkpoint duration + maximum restart duration or data loss may happen when Pulsar + expires an uncommitted transaction. + + See PulsarSinkBuilder for more details. + """ + + def __init__(self, j_pulsar_sink): + super(PulsarSink, self).__init__(sink=j_pulsar_sink) + + @staticmethod + def builder() -> 'PulsarSinkBuilder': + """ + Get a PulsarSinkBuilder to builder a PulsarSink. + """ + return PulsarSinkBuilder() + + +class PulsarSinkBuilder(object): + """ + The builder class for PulsarSink to make it easier for the users to construct a PulsarSink. + + The following example shows the minimum setup to create a PulsarSink that reads the String + values from a Pulsar topic. + + Example: + :: + + >>> sink = PulsarSink.builder() \\ + ... .set_service_url(PULSAR_BROKER_URL) \\ + ... .set_admin_url(PULSAR_BROKER_HTTP_URL) \\ + ... .set_topics([TOPIC1, TOPIC2]) \\ + ... .set_serialization_schema( + ... PulsarSerializationSchema.flink_schema(SimpleStringSchema())) \\ + ... .build() + + The service url, admin url, and the record serializer are required fields that must be set. If + you don't set the topics, make sure you have provided a custom TopicRouter. Otherwise, + you must provide the topics to produce. + + To specify the delivery guarantees of PulsarSink, one can call + #setDeliveryGuarantee(DeliveryGuarantee). The default value of the delivery guarantee is + DeliveryGuarantee#NONE, and it wouldn't promise the consistence when write the message into + Pulsar. + + Example: + :: + + >>> sink = PulsarSink.builder() \\ + ... .set_service_url(PULSAR_BROKER_URL) \\ + ... .set_admin_url(PULSAR_BROKER_HTTP_URL) \\ + ... .set_topics([TOPIC1, TOPIC2]) \\ + ... .set_serialization_schema( + ... PulsarSerializationSchema.flink_schema(SimpleStringSchema())) \\ + ... .set_delivery_guarantee(DeliveryGuarantee.EXACTLY_ONCE) + ... .build() + """ + + def __init__(self): + JPulsarSink = get_gateway().jvm.org.apache.flink.connector.pulsar.sink.PulsarSink + self._j_pulsar_sink_builder = JPulsarSink.builder() + + def set_admin_url(self, admin_url: str) -> 'PulsarSinkBuilder': + """ + Sets the admin endpoint for the PulsarAdmin of the PulsarSink. + """ + self._j_pulsar_sink_builder.setAdminUrl(admin_url) + return self + + def set_service_url(self, service_url: str) -> 'PulsarSinkBuilder': + """ + Sets the server's link for the PulsarProducer of the PulsarSink. + """ + self._j_pulsar_sink_builder.setServiceUrl(service_url) + return self + + def set_producer_name(self, producer_name: str) -> 'PulsarSinkBuilder': + """ + The producer name is informative, and it can be used to identify a particular producer + instance from the topic stats. + """ + self._j_pulsar_sink_builder.setProducerName(producer_name) + return self + + def set_topics(self, topics: Union[str, List[str]]) -> 'PulsarSinkBuilder': + """ + Set a pulsar topic list for flink sink. Some topic may not exist currently, write to this + non-existed topic wouldn't throw any exception. + """ + if not isinstance(topics, list): + topics = [topics] + self._j_pulsar_sink_builder.setTopics(topics) + return self + + def set_delivery_guarantee(self, delivery_guarantee: DeliveryGuarantee) -> 'PulsarSinkBuilder': + """ + Sets the wanted the DeliveryGuarantee. The default delivery guarantee is + DeliveryGuarantee#NONE. + """ + self._j_pulsar_sink_builder.setDeliveryGuarantee( + delivery_guarantee._to_j_delivery_guarantee()) + return self + + def set_topic_routing_mode(self, topic_routing_mode: TopicRoutingMode) -> 'PulsarSinkBuilder': + """ + Set a routing mode for choosing right topic partition to send messages. + """ + self._j_pulsar_sink_builder.setTopicRoutingMode( + topic_routing_mode._to_j_topic_routing_mode()) + return self + + def set_topic_router(self, topic_router_class_name: str) -> 'PulsarSinkBuilder': + """ + Use a custom topic router instead predefine topic routing. + """ + j_topic_router = load_java_class(topic_router_class_name).newInstance() + self._j_pulsar_sink_builder.setTopicRouter(j_topic_router) + return self + + def set_serialization_schema(self, pulsar_serialization_schema: PulsarSerializationSchema) \ + -> 'PulsarSinkBuilder': + """ + Sets the PulsarSerializationSchema that transforms incoming records to bytes. + """ + self._j_pulsar_sink_builder.setSerializationSchema( + pulsar_serialization_schema._j_pulsar_serialization_schema) + return self + + def delay_sending_message(self, message_delayer: MessageDelayer) -> 'PulsarSinkBuilder': + """ + Set a message delayer for enable Pulsar message delay delivery. + """ + self._j_pulsar_sink_builder.delaySendingMessage(message_delayer._j_message_delayer) + return self + + def set_config(self, key: str, value) -> 'PulsarSinkBuilder': + """ + Set an arbitrary property for the PulsarSink and Pulsar Producer. The valid keys can be + found in PulsarSinkOptions and PulsarOptions. + + Make sure the option could be set only once or with same value. + """ + j_config_option = ConfigOptions.key(key).string_type().no_default_value()._j_config_option + self._j_pulsar_sink_builder.setConfig(j_config_option, value) + return self + + def set_properties(self, config: Dict) -> 'PulsarSinkBuilder': + """ + Set an arbitrary property for the PulsarSink and Pulsar Producer. The valid keys can be + found in PulsarSinkOptions and PulsarOptions. + """ + JConfiguration = get_gateway().jvm.org.apache.flink.configuration.Configuration + self._j_pulsar_sink_builder.setConfig(JConfiguration.fromMap(config)) + return self + + def build(self) -> 'PulsarSink': + """ + Build the PulsarSink. + """ + return PulsarSink(self._j_pulsar_sink_builder.build()) diff --git a/flink-python/pyflink/datastream/stream_execution_environment.py b/flink-python/pyflink/datastream/stream_execution_environment.py index 545d2b3e0185b..d6469310cecec 100644 --- a/flink-python/pyflink/datastream/stream_execution_environment.py +++ b/flink-python/pyflink/datastream/stream_execution_environment.py @@ -964,7 +964,7 @@ def startup_loopback_server(): BeamFnLoopbackWorkerPoolServicer config = Configuration(j_configuration=j_configuration) config.set_string( - "PYFLINK_LOOPBACK_SERVER_ADDRESS", BeamFnLoopbackWorkerPoolServicer().start()) + "python.loopback-server.address", BeamFnLoopbackWorkerPoolServicer().start()) python_worker_execution_mode = os.environ.get('_python_worker_execution_mode') diff --git a/flink-python/pyflink/datastream/tests/test_data_stream.py b/flink-python/pyflink/datastream/tests/test_data_stream.py index e507468545abb..aa15dc21af7f2 100644 --- a/flink-python/pyflink/datastream/tests/test_data_stream.py +++ b/flink-python/pyflink/datastream/tests/test_data_stream.py @@ -791,7 +791,25 @@ def test_time_window(self): self.env.execute('test_time_window') results = self.test_sink.get_results() - expected = ['(hi,1)', '(hi,1)', '(hi,2)', '(hi,3)'] + expected = ['(hi,1)', '(hi,3)', '(hi,3)'] + self.assert_equals_sorted(expected, results) + + def test_session_window_late_merge(self): + self.env.set_parallelism(1) + data_stream = self.env.from_collection([ + ('hi', 0), ('hi', 2), ('hi', 1)], + type_info=Types.TUPLE([Types.STRING(), Types.INT()])) # type: DataStream + watermark_strategy = WatermarkStrategy.for_monotonous_timestamps() \ + .with_timestamp_assigner(SecondColumnTimestampAssigner()) + data_stream.assign_timestamps_and_watermarks(watermark_strategy) \ + .key_by(lambda x: x[0], key_type=Types.STRING()) \ + .window(SimpleMergeTimeWindowAssigner()) \ + .process(CountWindowProcessFunction(), Types.TUPLE([Types.STRING(), Types.INT()])) \ + .add_sink(self.test_sink) + + self.env.execute('test_session_window_late_merge') + results = self.test_sink.get_results() + expected = ['(hi,3)'] self.assert_equals_sorted(expected, results) @@ -1425,12 +1443,31 @@ class SimpleMergeTimeWindowAssigner(MergingWindowAssigner[tuple, TimeWindow]): def merge_windows(self, windows: Iterable[TimeWindow], callback: 'MergingWindowAssigner.MergeCallback[TimeWindow]') -> None: - window_list = [w for w in windows] - window_list.sort() - for i in range(1, len(window_list)): - if window_list[i - 1].end > window_list[i].start: - callback.merge([window_list[i - 1], window_list[i]], - TimeWindow(window_list[i - 1].start, window_list[i].end)) + sorted_windows = list(windows) + sorted_windows.sort() + merged = [] + current_merge = None + current_merge_set = set() + + for candidate in sorted_windows: + if current_merge is None: + current_merge = candidate + current_merge_set.add(candidate) + elif current_merge.intersects(candidate): + current_merge = current_merge.cover(candidate) + current_merge_set.add(candidate) + else: + merged.append((current_merge, current_merge_set)) + current_merge = candidate + current_merge_set = set() + current_merge_set.add(candidate) + + if current_merge is not None: + merged.append((current_merge, current_merge_set)) + + for merge_key, merge_set in merged: + if len(merge_set) > 1: + callback.merge(merge_set, merge_key) def assign_windows(self, element: tuple, diff --git a/flink-python/pyflink/datastream/tests/test_stream_execution_environment.py b/flink-python/pyflink/datastream/tests/test_stream_execution_environment.py index dd5dc475ffa33..16b129f4505f1 100644 --- a/flink-python/pyflink/datastream/tests/test_stream_execution_environment.py +++ b/flink-python/pyflink/datastream/tests/test_stream_execution_environment.py @@ -445,10 +445,10 @@ def plus_three(value): from test_dep2 import add_three return add_three(value) + env.add_python_file(python_file_path) t_env = StreamTableEnvironment.create( stream_execution_environment=env, environment_settings=EnvironmentSettings.in_streaming_mode()) - env.add_python_file(python_file_path) from pyflink.table.udf import udf from pyflink.table.expressions import col @@ -678,13 +678,15 @@ def add_from_file(i): # The parallelism of Sink: Test Sink should be 4 self.assertEqual(nodes[4]['parallelism'], 4) - env_config_with_dependencies = dict(get_gateway().jvm.org.apache.flink.python.util - .PythonConfigUtil.getEnvConfigWithDependencies( - env._j_stream_execution_environment).toMap()) + python_dependency_config = dict( + get_gateway().jvm.org.apache.flink.python.util.PythonDependencyUtils. + configurePythonDependencies( + env._j_stream_execution_environment.getCachedFiles(), + env._j_stream_execution_environment.getConfiguration()).toMap()) # Make sure that user specified files and archives are correctly added. - self.assertIsNotNone(env_config_with_dependencies['python.files']) - self.assertIsNotNone(env_config_with_dependencies['python.archives']) + self.assertIsNotNone(python_dependency_config['python.internal.files-key-map']) + self.assertIsNotNone(python_dependency_config['python.internal.archives-key-map']) def test_register_slot_sharing_group(self): slot_sharing_group_1 = SlotSharingGroup.builder('slot_sharing_group_1') \ diff --git a/flink-python/pyflink/datastream/window.py b/flink-python/pyflink/datastream/window.py index 53266e0051f8e..f310030ae14ba 100644 --- a/flink-python/pyflink/datastream/window.py +++ b/flink-python/pyflink/datastream/window.py @@ -15,12 +15,12 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ -import sys from abc import ABC, abstractmethod from enum import Enum from io import BytesIO from typing import TypeVar, Generic, Iterable, Collection +from pyflink.common.constants import MAX_LONG_VALUE from pyflink.common.serializer import TypeSerializer from pyflink.datastream.functions import RuntimeContext, InternalWindowFunction from pyflink.datastream.state import StateDescriptor, State @@ -36,8 +36,6 @@ 'TimeWindowSerializer', 'CountWindowSerializer'] -MAX_LONG_VALUE = sys.maxsize - def long_to_int_with_bit_mixing(x: int) -> int: x = (x ^ (x >> 30)) * 0xbf58476d1ce4e5b9 diff --git a/flink-python/pyflink/examples/datastream/basic_operations.py b/flink-python/pyflink/examples/datastream/basic_operations.py new file mode 100644 index 0000000000000..44f0fb864f51d --- /dev/null +++ b/flink-python/pyflink/examples/datastream/basic_operations.py @@ -0,0 +1,76 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ +import json +import logging +import sys + +from pyflink.common import Types +from pyflink.datastream import StreamExecutionEnvironment + + +def show(ds, env): + ds.print() + env.execute() + + +def basic_operations(): + env = StreamExecutionEnvironment.get_execution_environment() + env.set_parallelism(1) + + # define the source + ds = env.from_collection( + collection=[ + (1, '{"name": "Flink", "tel": 123, "addr": {"country": "Germany", "city": "Berlin"}}'), + (2, '{"name": "hello", "tel": 135, "addr": {"country": "China", "city": "Shanghai"}}'), + (3, '{"name": "world", "tel": 124, "addr": {"country": "USA", "city": "NewYork"}}'), + (4, '{"name": "PyFlink", "tel": 32, "addr": {"country": "China", "city": "Hangzhou"}}') + ], + type_info=Types.ROW_NAMED(["id", "info"], [Types.INT(), Types.STRING()]) + ) + + # map + def update_tel(data): + # parse the json + json_data = json.loads(data.info) + json_data['tel'] += 1 + return data.id, json.dumps(json_data) + + show(ds.map(update_tel), env) + # (1, '{"name": "Flink", "tel": 124, "addr": {"country": "Germany", "city": "Berlin"}}') + # (2, '{"name": "hello", "tel": 136, "addr": {"country": "China", "city": "Shanghai"}}') + # (3, '{"name": "world", "tel": 125, "addr": {"country": "USA", "city": "NewYork"}}') + # (4, '{"name": "PyFlink", "tel": 33, "addr": {"country": "China", "city": "Hangzhou"}}') + + # filter + show(ds.filter(lambda data: data.id == 1).map(update_tel), env) + # (1, '{"name": "Flink", "tel": 124, "addr": {"country": "Germany", "city": "Berlin"}}') + + # key by + show(ds.map(lambda data: (json.loads(data.info)['addr']['country'], + json.loads(data.info)['tel'])) + .key_by(lambda data: data[0]).reduce(lambda a, b: (a[0], a[1] + b[1])), env) + # ('Germany', 123) + # ('China', 135) + # ('USA', 124) + # ('China', 167) + + +if __name__ == '__main__': + logging.basicConfig(stream=sys.stdout, level=logging.INFO, format="%(message)s") + + basic_operations() diff --git a/flink-python/pyflink/examples/table/basic_operations.py b/flink-python/pyflink/examples/table/basic_operations.py new file mode 100644 index 0000000000000..6b9454b757db9 --- /dev/null +++ b/flink-python/pyflink/examples/table/basic_operations.py @@ -0,0 +1,445 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ +import json +import logging +import sys + +from pyflink.common import Row +from pyflink.table import (DataTypes, TableEnvironment, EnvironmentSettings) +from pyflink.table.expressions import * +from pyflink.table.udf import udtf, udf, udaf, AggregateFunction, TableAggregateFunction, udtaf + + +def basic_operations(): + t_env = TableEnvironment.create(EnvironmentSettings.in_streaming_mode()) + + # define the source + table = t_env.from_elements( + elements=[ + (1, '{"name": "Flink", "tel": 123, "addr": {"country": "Germany", "city": "Berlin"}}'), + (2, '{"name": "hello", "tel": 135, "addr": {"country": "China", "city": "Shanghai"}}'), + (3, '{"name": "world", "tel": 124, "addr": {"country": "USA", "city": "NewYork"}}'), + (4, '{"name": "PyFlink", "tel": 32, "addr": {"country": "China", "city": "Hangzhou"}}') + ], + schema=['id', 'data']) + + right_table = t_env.from_elements(elements=[(1, 18), (2, 30), (3, 25), (4, 10)], + schema=['id', 'age']) + + table = table.add_columns( + col('data').json_value('$.name', DataTypes.STRING()).alias('name'), + col('data').json_value('$.tel', DataTypes.STRING()).alias('tel'), + col('data').json_value('$.addr.country', DataTypes.STRING()).alias('country')) \ + .drop_columns(col('data')) + table.execute().print() + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + # | op | id | name | tel | country | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + # | +I | 1 | Flink | 123 | Germany | + # | +I | 2 | hello | 135 | China | + # | +I | 3 | world | 124 | USA | + # | +I | 4 | PyFlink | 32 | China | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + + # limit the number of outputs + table.limit(3).execute().print() + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + # | op | id | name | tel | country | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + # | +I | 1 | Flink | 123 | Germany | + # | +I | 2 | hello | 135 | China | + # | +I | 3 | world | 124 | USA | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + + # filter + table.filter(col('id') != 3).execute().print() + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + # | op | id | name | tel | country | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + # | +I | 1 | Flink | 123 | Germany | + # | +I | 2 | hello | 135 | China | + # | +I | 4 | PyFlink | 32 | China | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + + # aggregation + table.group_by(col('country')) \ + .select(col('country'), col('id').count, col('tel').cast(DataTypes.BIGINT()).max) \ + .execute().print() + # +----+--------------------------------+----------------------+----------------------+ + # | op | country | EXPR$0 | EXPR$1 | + # +----+--------------------------------+----------------------+----------------------+ + # | +I | Germany | 1 | 123 | + # | +I | USA | 1 | 124 | + # | +I | China | 1 | 135 | + # | -U | China | 1 | 135 | + # | +U | China | 2 | 135 | + # +----+--------------------------------+----------------------+----------------------+ + + # distinct + table.select(col('country')).distinct() \ + .execute().print() + # +----+--------------------------------+ + # | op | country | + # +----+--------------------------------+ + # | +I | Germany | + # | +I | China | + # | +I | USA | + # +----+--------------------------------+ + + # join + # Note that it still doesn't support duplicate column names between the joined tables + table.join(right_table.rename_columns(col('id').alias('r_id')), col('id') == col('r_id')) \ + .execute().print() + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+----------------------+----------------------+ + # | op | id | name | tel | country | r_id | age | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+----------------------+----------------------+ + # | +I | 4 | PyFlink | 32 | China | 4 | 10 | + # | +I | 1 | Flink | 123 | Germany | 1 | 18 | + # | +I | 2 | hello | 135 | China | 2 | 30 | + # | +I | 3 | world | 124 | USA | 3 | 25 | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+----------------------+----------------------+ + + # join lateral + @udtf(result_types=[DataTypes.STRING()]) + def split(r: Row): + for s in r.name.split("i"): + yield s + + table.join_lateral(split.alias('a')) \ + .execute().print() + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+--------------------------------+ + # | op | id | name | tel | country | a | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+--------------------------------+ + # | +I | 1 | Flink | 123 | Germany | Fl | + # | +I | 1 | Flink | 123 | Germany | nk | + # | +I | 2 | hello | 135 | China | hello | + # | +I | 3 | world | 124 | USA | world | + # | +I | 4 | PyFlink | 32 | China | PyFl | + # | +I | 4 | PyFlink | 32 | China | nk | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+--------------------------------+ + + # show schema + table.print_schema() + # ( + # `id` BIGINT, + # `name` STRING, + # `tel` STRING, + # `country` STRING + # ) + + # show execute plan + print(table.join_lateral(split.alias('a')).explain()) + # == Abstract Syntax Tree == + # LogicalCorrelate(correlation=[$cor1], joinType=[inner], requiredColumns=[{}]) + # :- LogicalProject(id=[$0], name=[JSON_VALUE($1, _UTF-16LE'$.name', FLAG(NULL), FLAG(ON EMPTY), FLAG(NULL), FLAG(ON ERROR))], tel=[JSON_VALUE($1, _UTF-16LE'$.tel', FLAG(NULL), FLAG(ON EMPTY), FLAG(NULL), FLAG(ON ERROR))], country=[JSON_VALUE($1, _UTF-16LE'$.addr.country', FLAG(NULL), FLAG(ON EMPTY), FLAG(NULL), FLAG(ON ERROR))]) + # : +- LogicalTableScan(table=[[default_catalog, default_database, Unregistered_TableSource_249535355, source: [PythonInputFormatTableSource(id, data)]]]) + # +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.functions.python.PythonTableFunction$1f0568d1f39bef59b4c969a5d620ba46*($0, $1, $2, $3)], rowType=[RecordType(VARCHAR(2147483647) a)], elementType=[class [Ljava.lang.Object;]) + # + # == Optimized Physical Plan == + # PythonCorrelate(invocation=[*org.apache.flink.table.functions.python.PythonTableFunction$1f0568d1f39bef59b4c969a5d620ba46*($0, $1, $2, $3)], correlate=[table(split(id,name,tel,country))], select=[id,name,tel,country,a], rowType=[RecordType(BIGINT id, VARCHAR(2147483647) name, VARCHAR(2147483647) tel, VARCHAR(2147483647) country, VARCHAR(2147483647) a)], joinType=[INNER]) + # +- Calc(select=[id, JSON_VALUE(data, _UTF-16LE'$.name', FLAG(NULL), FLAG(ON EMPTY), FLAG(NULL), FLAG(ON ERROR)) AS name, JSON_VALUE(data, _UTF-16LE'$.tel', FLAG(NULL), FLAG(ON EMPTY), FLAG(NULL), FLAG(ON ERROR)) AS tel, JSON_VALUE(data, _UTF-16LE'$.addr.country', FLAG(NULL), FLAG(ON EMPTY), FLAG(NULL), FLAG(ON ERROR)) AS country]) + # +- LegacyTableSourceScan(table=[[default_catalog, default_database, Unregistered_TableSource_249535355, source: [PythonInputFormatTableSource(id, data)]]], fields=[id, data]) + # + # == Optimized Execution Plan == + # PythonCorrelate(invocation=[*org.apache.flink.table.functions.python.PythonTableFunction$1f0568d1f39bef59b4c969a5d620ba46*($0, $1, $2, $3)], correlate=[table(split(id,name,tel,country))], select=[id,name,tel,country,a], rowType=[RecordType(BIGINT id, VARCHAR(2147483647) name, VARCHAR(2147483647) tel, VARCHAR(2147483647) country, VARCHAR(2147483647) a)], joinType=[INNER]) + # +- Calc(select=[id, JSON_VALUE(data, '$.name', NULL, ON EMPTY, NULL, ON ERROR) AS name, JSON_VALUE(data, '$.tel', NULL, ON EMPTY, NULL, ON ERROR) AS tel, JSON_VALUE(data, '$.addr.country', NULL, ON EMPTY, NULL, ON ERROR) AS country]) + # +- LegacyTableSourceScan(table=[[default_catalog, default_database, Unregistered_TableSource_249535355, source: [PythonInputFormatTableSource(id, data)]]], fields=[id, data]) + + +def sql_operations(): + t_env = TableEnvironment.create(EnvironmentSettings.in_streaming_mode()) + + # define the source + table = t_env.from_elements( + elements=[ + (1, '{"name": "Flink", "tel": 123, "addr": {"country": "Germany", "city": "Berlin"}}'), + (2, '{"name": "hello", "tel": 135, "addr": {"country": "China", "city": "Shanghai"}}'), + (3, '{"name": "world", "tel": 124, "addr": {"country": "USA", "city": "NewYork"}}'), + (4, '{"name": "PyFlink", "tel": 32, "addr": {"country": "China", "city": "Hangzhou"}}') + ], + schema=['id', 'data']) + + t_env.sql_query("SELECT * FROM %s" % table) \ + .execute().print() + # +----+----------------------+--------------------------------+ + # | op | id | data | + # +----+----------------------+--------------------------------+ + # | +I | 1 | {"name": "Flink", "tel": 12... | + # | +I | 2 | {"name": "hello", "tel": 13... | + # | +I | 3 | {"name": "world", "tel": 12... | + # | +I | 4 | {"name": "PyFlink", "tel": ... | + # +----+----------------------+--------------------------------+ + + # execute sql statement + @udtf(result_types=[DataTypes.STRING(), DataTypes.INT(), DataTypes.STRING()]) + def parse_data(data: str): + json_data = json.loads(data) + yield json_data['name'], json_data['tel'], json_data['addr']['country'] + + t_env.create_temporary_function('parse_data', parse_data) + t_env.execute_sql( + """ + SELECT * + FROM %s, LATERAL TABLE(parse_data(`data`)) t(name, tel, country) + """ % table + ).print() + # +----+----------------------+--------------------------------+--------------------------------+-------------+--------------------------------+ + # | op | id | data | name | tel | country | + # +----+----------------------+--------------------------------+--------------------------------+-------------+--------------------------------+ + # | +I | 1 | {"name": "Flink", "tel": 12... | Flink | 123 | Germany | + # | +I | 2 | {"name": "hello", "tel": 13... | hello | 135 | China | + # | +I | 3 | {"name": "world", "tel": 12... | world | 124 | USA | + # | +I | 4 | {"name": "PyFlink", "tel": ... | PyFlink | 32 | China | + # +----+----------------------+--------------------------------+--------------------------------+-------------+--------------------------------+ + + # explain sql plan + print(t_env.explain_sql( + """ + SELECT * + FROM %s, LATERAL TABLE(parse_data(`data`)) t(name, tel, country) + """ % table + )) + # == Abstract Syntax Tree == + # LogicalProject(id=[$0], data=[$1], name=[$2], tel=[$3], country=[$4]) + # +- LogicalCorrelate(correlation=[$cor1], joinType=[inner], requiredColumns=[{1}]) + # :- LogicalTableScan(table=[[default_catalog, default_database, Unregistered_TableSource_734856049, source: [PythonInputFormatTableSource(id, data)]]]) + # +- LogicalTableFunctionScan(invocation=[parse_data($cor1.data)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) f0, INTEGER f1, VARCHAR(2147483647) f2)]) + # + # == Optimized Physical Plan == + # PythonCorrelate(invocation=[parse_data($1)], correlate=[table(parse_data(data))], select=[id,data,f0,f1,f2], rowType=[RecordType(BIGINT id, VARCHAR(2147483647) data, VARCHAR(2147483647) f0, INTEGER f1, VARCHAR(2147483647) f2)], joinType=[INNER]) + # +- LegacyTableSourceScan(table=[[default_catalog, default_database, Unregistered_TableSource_734856049, source: [PythonInputFormatTableSource(id, data)]]], fields=[id, data]) + # + # == Optimized Execution Plan == + # PythonCorrelate(invocation=[parse_data($1)], correlate=[table(parse_data(data))], select=[id,data,f0,f1,f2], rowType=[RecordType(BIGINT id, VARCHAR(2147483647) data, VARCHAR(2147483647) f0, INTEGER f1, VARCHAR(2147483647) f2)], joinType=[INNER]) + # +- LegacyTableSourceScan(table=[[default_catalog, default_database, Unregistered_TableSource_734856049, source: [PythonInputFormatTableSource(id, data)]]], fields=[id, data]) + + +def column_operations(): + t_env = TableEnvironment.create(EnvironmentSettings.in_streaming_mode()) + + # define the source + table = t_env.from_elements( + elements=[ + (1, '{"name": "Flink", "tel": 123, "addr": {"country": "Germany", "city": "Berlin"}}'), + (2, '{"name": "hello", "tel": 135, "addr": {"country": "China", "city": "Shanghai"}}'), + (3, '{"name": "world", "tel": 124, "addr": {"country": "USA", "city": "NewYork"}}'), + (4, '{"name": "PyFlink", "tel": 32, "addr": {"country": "China", "city": "Hangzhou"}}') + ], + schema=['id', 'data']) + + # add columns + table = table.add_columns( + col('data').json_value('$.name', DataTypes.STRING()).alias('name'), + col('data').json_value('$.tel', DataTypes.STRING()).alias('tel'), + col('data').json_value('$.addr.country', DataTypes.STRING()).alias('country')) + + table.execute().print() + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+--------------------------------+ + # | op | id | data | name | tel | country | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+--------------------------------+ + # | +I | 1 | {"name": "Flink", "tel": 12... | Flink | 123 | Germany | + # | +I | 2 | {"name": "hello", "tel": 13... | hello | 135 | China | + # | +I | 3 | {"name": "world", "tel": 12... | world | 124 | USA | + # | +I | 4 | {"name": "PyFlink", "tel": ... | PyFlink | 32 | China | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+--------------------------------+ + + # drop columns + table = table.drop_columns(col('data')) + table.execute().print() + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + # | op | id | name | tel | country | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + # | +I | 1 | Flink | 123 | Germany | + # | +I | 2 | hello | 135 | China | + # | +I | 3 | world | 124 | USA | + # | +I | 4 | PyFlink | 32 | China | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + + # rename columns + table = table.rename_columns(col('tel').alias('telephone')) + table.execute().print() + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + # | op | id | name | telephone | country | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + # | +I | 1 | Flink | 123 | Germany | + # | +I | 2 | hello | 135 | China | + # | +I | 3 | world | 124 | USA | + # | +I | 4 | PyFlink | 32 | China | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + + # replace columns + table = table.add_or_replace_columns( + concat(col('id').cast(DataTypes.STRING()), '_', col('name')).alias('id')) + table.execute().print() + # +----+--------------------------------+--------------------------------+--------------------------------+--------------------------------+ + # | op | id | name | telephone | country | + # +----+--------------------------------+--------------------------------+--------------------------------+--------------------------------+ + # | +I | 1_Flink | Flink | 123 | Germany | + # | +I | 2_hello | hello | 135 | China | + # | +I | 3_world | world | 124 | USA | + # | +I | 4_PyFlink | PyFlink | 32 | China | + # +----+--------------------------------+--------------------------------+--------------------------------+--------------------------------+ + + +def row_operations(): + t_env = TableEnvironment.create(EnvironmentSettings.in_streaming_mode()) + + # define the source + table = t_env.from_elements( + elements=[ + (1, '{"name": "Flink", "tel": 123, "addr": {"country": "Germany", "city": "Berlin"}}'), + (2, '{"name": "hello", "tel": 135, "addr": {"country": "China", "city": "Shanghai"}}'), + (3, '{"name": "world", "tel": 124, "addr": {"country": "China", "city": "NewYork"}}'), + (4, '{"name": "PyFlink", "tel": 32, "addr": {"country": "China", "city": "Hangzhou"}}') + ], + schema=['id', 'data']) + + # map operation + @udf(result_type=DataTypes.ROW([DataTypes.FIELD("id", DataTypes.BIGINT()), + DataTypes.FIELD("country", DataTypes.STRING())])) + def extract_country(input_row: Row): + data = json.loads(input_row.data) + return Row(input_row.id, data['addr']['country']) + + table.map(extract_country) \ + .execute().print() + # +----+----------------------+--------------------------------+ + # | op | _c0 | _c1 | + # +----+----------------------+--------------------------------+ + # | +I | 1 | Germany | + # | +I | 2 | China | + # | +I | 3 | China | + # | +I | 4 | China | + # +----+----------------------+--------------------------------+ + + # flat_map operation + @udtf(result_types=[DataTypes.BIGINT(), DataTypes.STRING()]) + def extract_city(input_row: Row): + data = json.loads(input_row.data) + yield input_row.id, data['addr']['city'] + + table.flat_map(extract_city) \ + .execute().print() + # +----+----------------------+--------------------------------+ + # | op | f0 | f1 | + # +----+----------------------+--------------------------------+ + # | +I | 1 | Berlin | + # | +I | 2 | Shanghai | + # | +I | 3 | NewYork | + # | +I | 4 | Hangzhou | + # +----+----------------------+--------------------------------+ + + # aggregate operation + class CountAndSumAggregateFunction(AggregateFunction): + + def get_value(self, accumulator): + return Row(accumulator[0], accumulator[1]) + + def create_accumulator(self): + return Row(0, 0) + + def accumulate(self, accumulator, input_row): + accumulator[0] += 1 + accumulator[1] += int(input_row.tel) + + def retract(self, accumulator, input_row): + accumulator[0] -= 1 + accumulator[1] -= int(input_row.tel) + + def merge(self, accumulator, accumulators): + for other_acc in accumulators: + accumulator[0] += other_acc[0] + accumulator[1] += other_acc[1] + + def get_accumulator_type(self): + return DataTypes.ROW( + [DataTypes.FIELD("cnt", DataTypes.BIGINT()), + DataTypes.FIELD("sum", DataTypes.BIGINT())]) + + def get_result_type(self): + return DataTypes.ROW( + [DataTypes.FIELD("cnt", DataTypes.BIGINT()), + DataTypes.FIELD("sum", DataTypes.BIGINT())]) + + count_sum = udaf(CountAndSumAggregateFunction()) + table.add_columns( + col('data').json_value('$.name', DataTypes.STRING()).alias('name'), + col('data').json_value('$.tel', DataTypes.STRING()).alias('tel'), + col('data').json_value('$.addr.country', DataTypes.STRING()).alias('country')) \ + .group_by(col('country')) \ + .aggregate(count_sum.alias("cnt", "sum")) \ + .select(col('country'), col('cnt'), col('sum')) \ + .execute().print() + # +----+--------------------------------+----------------------+----------------------+ + # | op | country | cnt | sum | + # +----+--------------------------------+----------------------+----------------------+ + # | +I | China | 3 | 291 | + # | +I | Germany | 1 | 123 | + # +----+--------------------------------+----------------------+----------------------+ + + # flat_aggregate operation + class Top2(TableAggregateFunction): + + def emit_value(self, accumulator): + for v in accumulator: + if v: + yield Row(v) + + def create_accumulator(self): + return [None, None] + + def accumulate(self, accumulator, input_row): + tel = int(input_row.tel) + if accumulator[0] is None or tel > accumulator[0]: + accumulator[1] = accumulator[0] + accumulator[0] = tel + elif accumulator[1] is None or tel > accumulator[1]: + accumulator[1] = tel + + def get_accumulator_type(self): + return DataTypes.ARRAY(DataTypes.BIGINT()) + + def get_result_type(self): + return DataTypes.ROW( + [DataTypes.FIELD("tel", DataTypes.BIGINT())]) + + top2 = udtaf(Top2()) + table.add_columns( + col('data').json_value('$.name', DataTypes.STRING()).alias('name'), + col('data').json_value('$.tel', DataTypes.STRING()).alias('tel'), + col('data').json_value('$.addr.country', DataTypes.STRING()).alias('country')) \ + .group_by(col('country')) \ + .flat_aggregate(top2) \ + .select(col('country'), col('tel')) \ + .execute().print() + # +----+--------------------------------+----------------------+ + # | op | country | tel | + # +----+--------------------------------+----------------------+ + # | +I | China | 135 | + # | +I | China | 124 | + # | +I | Germany | 123 | + # +----+--------------------------------+----------------------+ + + +if __name__ == '__main__': + logging.basicConfig(stream=sys.stdout, level=logging.INFO, format="%(message)s") + + basic_operations() + sql_operations() + column_operations() + row_operations() diff --git a/flink-python/pyflink/examples/table/pandas/conversion_from_dataframe.py b/flink-python/pyflink/examples/table/pandas/conversion_from_dataframe.py index 2911006becd70..361565f1efdb2 100644 --- a/flink-python/pyflink/examples/table/pandas/conversion_from_dataframe.py +++ b/flink-python/pyflink/examples/table/pandas/conversion_from_dataframe.py @@ -26,7 +26,7 @@ def conversion_from_dataframe(): t_env = TableEnvironment.create(EnvironmentSettings.in_streaming_mode()) - t_env.get_config().get_configuration().set_string("parallelism.default", "1") + t_env.get_config().set("parallelism.default", "1") # define the source with watermark definition pdf = pd.DataFrame(np.random.rand(1000, 2)) diff --git a/flink-python/pyflink/examples/table/word_count.py b/flink-python/pyflink/examples/table/word_count.py index 986a17d6f5d08..a0283048925c0 100644 --- a/flink-python/pyflink/examples/table/word_count.py +++ b/flink-python/pyflink/examples/table/word_count.py @@ -65,7 +65,7 @@ def word_count(input_path, output_path): t_env = TableEnvironment.create(EnvironmentSettings.in_streaming_mode()) # write all the data to one file - t_env.get_config().get_configuration().set_string("parallelism.default", "1") + t_env.get_config().set("parallelism.default", "1") # define the source if input_path is not None: diff --git a/flink-python/pyflink/fn_execution/beam/beam_boot.py b/flink-python/pyflink/fn_execution/beam/beam_boot.py index 411745c0c23e6..4908944864ce6 100644 --- a/flink-python/pyflink/fn_execution/beam/beam_boot.py +++ b/flink-python/pyflink/fn_execution/beam/beam_boot.py @@ -75,12 +75,12 @@ def check_not_empty(check_str, error_message): logging.info("Initializing Python harness: %s" % " ".join(sys.argv)) - if 'PYFLINK_LOOPBACK_SERVER_ADDRESS' in os.environ: + if 'PYTHON_LOOPBACK_SERVER_ADDRESS' in os.environ: logging.info("Starting up Python harness in loopback mode.") params = dict(os.environ) params.update({'SEMI_PERSISTENT_DIRECTORY': semi_persist_dir}) - with grpc.insecure_channel(os.environ['PYFLINK_LOOPBACK_SERVER_ADDRESS']) as channel: + with grpc.insecure_channel(os.environ['PYTHON_LOOPBACK_SERVER_ADDRESS']) as channel: client = BeamFnExternalWorkerPoolStub(channel=channel) request = StartWorkerRequest( worker_id=worker_id, diff --git a/flink-python/pyflink/fn_execution/beam/beam_operations.py b/flink-python/pyflink/fn_execution/beam/beam_operations.py index 17074b23c5b79..3777c26563880 100644 --- a/flink-python/pyflink/fn_execution/beam/beam_operations.py +++ b/flink-python/pyflink/fn_execution/beam/beam_operations.py @@ -185,9 +185,9 @@ def _create_user_defined_function_operation(factory, transform_proto, consumers, factory.state_handler, key_row_coder, None, - 1000, - 1000, - 1000) + serialized_fn.state_cache_size, + serialized_fn.map_state_read_cache_size, + serialized_fn.map_state_write_cache_size) return beam_operation_cls( transform_proto.unique_name, spec, diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index 2ddd2f30109f2..7e43f78b2b243 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -79,12 +79,12 @@ def _to_field_coder(cls, coder_info_descriptor_proto): field_names = [f.name for f in schema_proto.fields] return RowCoder(field_coders, field_names) elif coder_info_descriptor_proto.HasField('arrow_type'): - timezone = pytz.timezone(os.environ['table.exec.timezone']) + timezone = pytz.timezone(os.environ['TABLE_LOCAL_TIME_ZONE']) schema_proto = coder_info_descriptor_proto.arrow_type.schema row_type = cls._to_row_type(schema_proto) return ArrowCoder(cls._to_arrow_schema(row_type), row_type, timezone) elif coder_info_descriptor_proto.HasField('over_window_arrow_type'): - timezone = pytz.timezone(os.environ['table.exec.timezone']) + timezone = pytz.timezone(os.environ['TABLE_LOCAL_TIME_ZONE']) schema_proto = coder_info_descriptor_proto.over_window_arrow_type.schema row_type = cls._to_row_type(schema_proto) return OverWindowArrowCoder( @@ -633,7 +633,7 @@ def from_proto(field_type): if field_type_name == type_name.TIMESTAMP: return TimestampCoder(field_type.timestamp_info.precision) if field_type_name == type_name.LOCAL_ZONED_TIMESTAMP: - timezone = pytz.timezone(os.environ['table.exec.timezone']) + timezone = pytz.timezone(os.environ['TABLE_LOCAL_TIME_ZONE']) return LocalZonedTimestampCoder(field_type.local_zoned_timestamp_info.precision, timezone) elif field_type_name == type_name.BASIC_ARRAY: return GenericArrayCoder(from_proto(field_type.collection_element_type)) diff --git a/flink-python/pyflink/fn_execution/datastream/window/merging_window_set.py b/flink-python/pyflink/fn_execution/datastream/window/merging_window_set.py index 639a87e53db59..a805817713208 100644 --- a/flink-python/pyflink/fn_execution/datastream/window/merging_window_set.py +++ b/flink-python/pyflink/fn_execution/datastream/window/merging_window_set.py @@ -102,7 +102,7 @@ def add_window(self, new_window: W, merge_function: MergeFunction[W]): self._mapping[merge_result] = merged_state_window merged_state_windows.remove(merged_state_window) - if merge_result not in merged_windows and len(merged_windows) == 1: + if merge_result not in merged_windows or len(merged_windows) != 1: merge_function.merge( merge_result, merged_windows, diff --git a/flink-python/pyflink/fn_execution/datastream/window/window_operator.py b/flink-python/pyflink/fn_execution/datastream/window/window_operator.py index be172f20435d8..234bc97863b1e 100644 --- a/flink-python/pyflink/fn_execution/datastream/window/window_operator.py +++ b/flink-python/pyflink/fn_execution/datastream/window/window_operator.py @@ -18,13 +18,13 @@ import typing from typing import TypeVar, Iterable, Collection +from pyflink.common.constants import MAX_LONG_VALUE from pyflink.datastream import WindowAssigner, Trigger, MergingWindowAssigner, TriggerResult from pyflink.datastream.functions import KeyedStateStore, RuntimeContext, InternalWindowFunction from pyflink.datastream.state import StateDescriptor, ListStateDescriptor, \ ReducingStateDescriptor, AggregatingStateDescriptor, ValueStateDescriptor, MapStateDescriptor, \ State, AggregatingState, ReducingState, MapState, ListState, ValueState, AppendingState from pyflink.fn_execution.datastream.timerservice import InternalTimerService -from pyflink.datastream.window import MAX_LONG_VALUE from pyflink.fn_execution.datastream.window.merging_window_set import MergingWindowSet from pyflink.fn_execution.internal_state import InternalMergingState, InternalKvState, \ InternalAppendingState @@ -375,7 +375,7 @@ def process_element(self, value, timestamp: int): if trigger_result.is_purge(): self.window_state.clear() - self.register_cleanup_timer(window) + self.register_cleanup_timer(actual_window) merging_windows.persist() else: diff --git a/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py b/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py index c087b641bedcd..90b0bc5af53e8 100644 --- a/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py +++ b/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py @@ -36,7 +36,7 @@ name='flink-fn-execution.proto', package='org.apache.flink.fn_execution.v1', syntax='proto3', - serialized_pb=_b('\n\x18\x66link-fn-execution.proto\x12 org.apache.flink.fn_execution.v1\"\x86\x01\n\x05Input\x12\x44\n\x03udf\x18\x01 \x01(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunctionH\x00\x12\x15\n\x0binputOffset\x18\x02 \x01(\x05H\x00\x12\x17\n\rinputConstant\x18\x03 \x01(\x0cH\x00\x42\x07\n\x05input\"\xa8\x01\n\x13UserDefinedFunction\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12\x37\n\x06inputs\x18\x02 \x03(\x0b\x32\'.org.apache.flink.fn_execution.v1.Input\x12\x14\n\x0cwindow_index\x18\x03 \x01(\x05\x12\x1a\n\x12takes_row_as_input\x18\x04 \x01(\x08\x12\x15\n\ris_pandas_udf\x18\x05 \x01(\x08\"\xcb\x01\n\x14UserDefinedFunctions\x12\x43\n\x04udfs\x18\x01 \x03(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunction\x12\x16\n\x0emetric_enabled\x18\x02 \x01(\x08\x12=\n\x07windows\x18\x03 \x03(\x0b\x32,.org.apache.flink.fn_execution.v1.OverWindow\x12\x17\n\x0fprofile_enabled\x18\x04 \x01(\x08\"\xdd\x02\n\nOverWindow\x12L\n\x0bwindow_type\x18\x01 \x01(\x0e\x32\x37.org.apache.flink.fn_execution.v1.OverWindow.WindowType\x12\x16\n\x0elower_boundary\x18\x02 \x01(\x03\x12\x16\n\x0eupper_boundary\x18\x03 \x01(\x03\"\xd0\x01\n\nWindowType\x12\x13\n\x0fRANGE_UNBOUNDED\x10\x00\x12\x1d\n\x19RANGE_UNBOUNDED_PRECEDING\x10\x01\x12\x1d\n\x19RANGE_UNBOUNDED_FOLLOWING\x10\x02\x12\x11\n\rRANGE_SLIDING\x10\x03\x12\x11\n\rROW_UNBOUNDED\x10\x04\x12\x1b\n\x17ROW_UNBOUNDED_PRECEDING\x10\x05\x12\x1b\n\x17ROW_UNBOUNDED_FOLLOWING\x10\x06\x12\x0f\n\x0bROW_SLIDING\x10\x07\"\x8b\x06\n\x1cUserDefinedAggregateFunction\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12\x37\n\x06inputs\x18\x02 \x03(\x0b\x32\'.org.apache.flink.fn_execution.v1.Input\x12Z\n\x05specs\x18\x03 \x03(\x0b\x32K.org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec\x12\x12\n\nfilter_arg\x18\x04 \x01(\x05\x12\x10\n\x08\x64istinct\x18\x05 \x01(\x08\x12\x1a\n\x12takes_row_as_input\x18\x06 \x01(\x08\x1a\x82\x04\n\x0c\x44\x61taViewSpec\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x13\n\x0b\x66ield_index\x18\x02 \x01(\x05\x12i\n\tlist_view\x18\x03 \x01(\x0b\x32T.org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.ListViewH\x00\x12g\n\x08map_view\x18\x04 \x01(\x0b\x32S.org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.MapViewH\x00\x1aT\n\x08ListView\x12H\n\x0c\x65lement_type\x18\x01 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x1a\x97\x01\n\x07MapView\x12\x44\n\x08key_type\x18\x01 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x12\x46\n\nvalue_type\x18\x02 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldTypeB\x0b\n\tdata_view\"\xac\x04\n\x0bGroupWindow\x12M\n\x0bwindow_type\x18\x01 \x01(\x0e\x32\x38.org.apache.flink.fn_execution.v1.GroupWindow.WindowType\x12\x16\n\x0eis_time_window\x18\x02 \x01(\x08\x12\x14\n\x0cwindow_slide\x18\x03 \x01(\x03\x12\x13\n\x0bwindow_size\x18\x04 \x01(\x03\x12\x12\n\nwindow_gap\x18\x05 \x01(\x03\x12\x13\n\x0bis_row_time\x18\x06 \x01(\x08\x12\x18\n\x10time_field_index\x18\x07 \x01(\x05\x12\x17\n\x0f\x61llowedLateness\x18\x08 \x01(\x03\x12U\n\x0fnamedProperties\x18\t \x03(\x0e\x32<.org.apache.flink.fn_execution.v1.GroupWindow.WindowProperty\x12\x16\n\x0eshift_timezone\x18\n \x01(\t\"[\n\nWindowType\x12\x19\n\x15TUMBLING_GROUP_WINDOW\x10\x00\x12\x18\n\x14SLIDING_GROUP_WINDOW\x10\x01\x12\x18\n\x14SESSION_GROUP_WINDOW\x10\x02\"c\n\x0eWindowProperty\x12\x10\n\x0cWINDOW_START\x10\x00\x12\x0e\n\nWINDOW_END\x10\x01\x12\x16\n\x12ROW_TIME_ATTRIBUTE\x10\x02\x12\x17\n\x13PROC_TIME_ATTRIBUTE\x10\x03\"\x96\x04\n\x1dUserDefinedAggregateFunctions\x12L\n\x04udfs\x18\x01 \x03(\x0b\x32>.org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction\x12\x16\n\x0emetric_enabled\x18\x02 \x01(\x08\x12\x10\n\x08grouping\x18\x03 \x03(\x05\x12\x1e\n\x16generate_update_before\x18\x04 \x01(\x08\x12\x44\n\x08key_type\x18\x05 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x12\x1b\n\x13index_of_count_star\x18\x06 \x01(\x05\x12\x1e\n\x16state_cleaning_enabled\x18\x07 \x01(\x08\x12\x18\n\x10state_cache_size\x18\x08 \x01(\x05\x12!\n\x19map_state_read_cache_size\x18\t \x01(\x05\x12\"\n\x1amap_state_write_cache_size\x18\n \x01(\x05\x12\x1b\n\x13\x63ount_star_inserted\x18\x0b \x01(\x08\x12\x43\n\x0cgroup_window\x18\x0c \x01(\x0b\x32-.org.apache.flink.fn_execution.v1.GroupWindow\x12\x17\n\x0fprofile_enabled\x18\r \x01(\x08\"\xec\x0f\n\x06Schema\x12>\n\x06\x66ields\x18\x01 \x03(\x0b\x32..org.apache.flink.fn_execution.v1.Schema.Field\x1a\x97\x01\n\x07MapInfo\x12\x44\n\x08key_type\x18\x01 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x12\x46\n\nvalue_type\x18\x02 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x1a\x1d\n\x08TimeInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a\"\n\rTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a,\n\x17LocalZonedTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a\'\n\x12ZonedTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a/\n\x0b\x44\x65\x63imalInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x12\r\n\x05scale\x18\x02 \x01(\x05\x1a\x1c\n\nBinaryInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1f\n\rVarBinaryInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1a\n\x08\x43harInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1d\n\x0bVarCharInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\xb0\x08\n\tFieldType\x12\x44\n\ttype_name\x18\x01 \x01(\x0e\x32\x31.org.apache.flink.fn_execution.v1.Schema.TypeName\x12\x10\n\x08nullable\x18\x02 \x01(\x08\x12U\n\x17\x63ollection_element_type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldTypeH\x00\x12\x44\n\x08map_info\x18\x04 \x01(\x0b\x32\x30.org.apache.flink.fn_execution.v1.Schema.MapInfoH\x00\x12>\n\nrow_schema\x18\x05 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.SchemaH\x00\x12L\n\x0c\x64\x65\x63imal_info\x18\x06 \x01(\x0b\x32\x34.org.apache.flink.fn_execution.v1.Schema.DecimalInfoH\x00\x12\x46\n\ttime_info\x18\x07 \x01(\x0b\x32\x31.org.apache.flink.fn_execution.v1.Schema.TimeInfoH\x00\x12P\n\x0etimestamp_info\x18\x08 \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.Schema.TimestampInfoH\x00\x12\x66\n\x1alocal_zoned_timestamp_info\x18\t \x01(\x0b\x32@.org.apache.flink.fn_execution.v1.Schema.LocalZonedTimestampInfoH\x00\x12[\n\x14zoned_timestamp_info\x18\n \x01(\x0b\x32;.org.apache.flink.fn_execution.v1.Schema.ZonedTimestampInfoH\x00\x12J\n\x0b\x62inary_info\x18\x0b \x01(\x0b\x32\x33.org.apache.flink.fn_execution.v1.Schema.BinaryInfoH\x00\x12Q\n\x0fvar_binary_info\x18\x0c \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.Schema.VarBinaryInfoH\x00\x12\x46\n\tchar_info\x18\r \x01(\x0b\x32\x31.org.apache.flink.fn_execution.v1.Schema.CharInfoH\x00\x12M\n\rvar_char_info\x18\x0e \x01(\x0b\x32\x34.org.apache.flink.fn_execution.v1.Schema.VarCharInfoH\x00\x42\x0b\n\ttype_info\x1al\n\x05\x46ield\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x02 \x01(\t\x12@\n\x04type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\"\xa1\x02\n\x08TypeName\x12\x07\n\x03ROW\x10\x00\x12\x0b\n\x07TINYINT\x10\x01\x12\x0c\n\x08SMALLINT\x10\x02\x12\x07\n\x03INT\x10\x03\x12\n\n\x06\x42IGINT\x10\x04\x12\x0b\n\x07\x44\x45\x43IMAL\x10\x05\x12\t\n\x05\x46LOAT\x10\x06\x12\n\n\x06\x44OUBLE\x10\x07\x12\x08\n\x04\x44\x41TE\x10\x08\x12\x08\n\x04TIME\x10\t\x12\r\n\tTIMESTAMP\x10\n\x12\x0b\n\x07\x42OOLEAN\x10\x0b\x12\n\n\x06\x42INARY\x10\x0c\x12\r\n\tVARBINARY\x10\r\x12\x08\n\x04\x43HAR\x10\x0e\x12\x0b\n\x07VARCHAR\x10\x0f\x12\x0f\n\x0b\x42\x41SIC_ARRAY\x10\x10\x12\x07\n\x03MAP\x10\x11\x12\x0c\n\x08MULTISET\x10\x12\x12\x19\n\x15LOCAL_ZONED_TIMESTAMP\x10\x13\x12\x13\n\x0fZONED_TIMESTAMP\x10\x14\"\xf7\x08\n\x08TypeInfo\x12\x46\n\ttype_name\x18\x01 \x01(\x0e\x32\x33.org.apache.flink.fn_execution.v1.TypeInfo.TypeName\x12M\n\x17\x63ollection_element_type\x18\x02 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfoH\x00\x12O\n\rrow_type_info\x18\x03 \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.TypeInfo.RowTypeInfoH\x00\x12S\n\x0ftuple_type_info\x18\x04 \x01(\x0b\x32\x38.org.apache.flink.fn_execution.v1.TypeInfo.TupleTypeInfoH\x00\x12O\n\rmap_type_info\x18\x05 \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.TypeInfo.MapTypeInfoH\x00\x1a\x8b\x01\n\x0bMapTypeInfo\x12<\n\x08key_type\x18\x01 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\x12>\n\nvalue_type\x18\x02 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\x1a\xb8\x01\n\x0bRowTypeInfo\x12L\n\x06\x66ields\x18\x01 \x03(\x0b\x32<.org.apache.flink.fn_execution.v1.TypeInfo.RowTypeInfo.Field\x1a[\n\x05\x46ield\x12\x12\n\nfield_name\x18\x01 \x01(\t\x12>\n\nfield_type\x18\x02 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\x1aP\n\rTupleTypeInfo\x12?\n\x0b\x66ield_types\x18\x01 \x03(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\"\xb4\x02\n\x08TypeName\x12\x07\n\x03ROW\x10\x00\x12\n\n\x06STRING\x10\x01\x12\x08\n\x04\x42YTE\x10\x02\x12\x0b\n\x07\x42OOLEAN\x10\x03\x12\t\n\x05SHORT\x10\x04\x12\x07\n\x03INT\x10\x05\x12\x08\n\x04LONG\x10\x06\x12\t\n\x05\x46LOAT\x10\x07\x12\n\n\x06\x44OUBLE\x10\x08\x12\x08\n\x04\x43HAR\x10\t\x12\x0b\n\x07\x42IG_INT\x10\n\x12\x0b\n\x07\x42IG_DEC\x10\x0b\x12\x0c\n\x08SQL_DATE\x10\x0c\x12\x0c\n\x08SQL_TIME\x10\r\x12\x11\n\rSQL_TIMESTAMP\x10\x0e\x12\x0f\n\x0b\x42\x41SIC_ARRAY\x10\x0f\x12\x13\n\x0fPRIMITIVE_ARRAY\x10\x10\x12\t\n\x05TUPLE\x10\x11\x12\x08\n\x04LIST\x10\x12\x12\x07\n\x03MAP\x10\x13\x12\x11\n\rPICKLED_BYTES\x10\x14\x12\x10\n\x0cOBJECT_ARRAY\x10\x15\x12\x0b\n\x07INSTANT\x10\x16\x42\x0b\n\ttype_info\"\xe6\x06\n\x1dUserDefinedDataStreamFunction\x12\x63\n\rfunction_type\x18\x01 \x01(\x0e\x32L.org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.FunctionType\x12g\n\x0fruntime_context\x18\x02 \x01(\x0b\x32N.org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext\x12\x0f\n\x07payload\x18\x03 \x01(\x0c\x12\x16\n\x0emetric_enabled\x18\x04 \x01(\x08\x12\x41\n\rkey_type_info\x18\x05 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\x12\x17\n\x0fprofile_enabled\x18\x06 \x01(\x08\x1a*\n\x0cJobParameter\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t\x1a\xd0\x02\n\x0eRuntimeContext\x12\x11\n\ttask_name\x18\x01 \x01(\t\x12\x1f\n\x17task_name_with_subtasks\x18\x02 \x01(\t\x12#\n\x1bnumber_of_parallel_subtasks\x18\x03 \x01(\x05\x12\'\n\x1fmax_number_of_parallel_subtasks\x18\x04 \x01(\x05\x12\x1d\n\x15index_of_this_subtask\x18\x05 \x01(\x05\x12\x16\n\x0e\x61ttempt_number\x18\x06 \x01(\x05\x12\x64\n\x0ejob_parameters\x18\x07 \x03(\x0b\x32L.org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.JobParameter\x12\x1f\n\x17in_batch_execution_mode\x18\x08 \x01(\x08\"s\n\x0c\x46unctionType\x12\x0b\n\x07PROCESS\x10\x00\x12\x0e\n\nCO_PROCESS\x10\x01\x12\x11\n\rKEYED_PROCESS\x10\x02\x12\x14\n\x10KEYED_CO_PROCESS\x10\x03\x12\n\n\x06WINDOW\x10\x04\x12\x11\n\rREVISE_OUTPUT\x10\x64\"\xe4\x0e\n\x0fStateDescriptor\x12\x12\n\nstate_name\x18\x01 \x01(\t\x12Z\n\x10state_ttl_config\x18\x02 \x01(\x0b\x32@.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig\x1a\xe0\r\n\x0eStateTTLConfig\x12`\n\x0bupdate_type\x18\x01 \x01(\x0e\x32K.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.UpdateType\x12j\n\x10state_visibility\x18\x02 \x01(\x0e\x32P.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.StateVisibility\x12w\n\x17ttl_time_characteristic\x18\x03 \x01(\x0e\x32V.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.TtlTimeCharacteristic\x12\x0b\n\x03ttl\x18\x04 \x01(\x03\x12n\n\x12\x63leanup_strategies\x18\x05 \x01(\x0b\x32R.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies\x1a\xca\x08\n\x11\x43leanupStrategies\x12 \n\x18is_cleanup_in_background\x18\x01 \x01(\x08\x12y\n\nstrategies\x18\x02 \x03(\x0b\x32\x65.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies.MapStrategiesEntry\x1aX\n\x1aIncrementalCleanupStrategy\x12\x14\n\x0c\x63leanup_size\x18\x01 \x01(\x05\x12$\n\x1crun_cleanup_for_every_record\x18\x02 \x01(\x08\x1aK\n#RocksdbCompactFilterCleanupStrategy\x12$\n\x1cquery_time_after_num_entries\x18\x01 \x01(\x03\x1a\xe0\x04\n\x12MapStrategiesEntry\x12o\n\x08strategy\x18\x01 \x01(\x0e\x32].org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies.Strategies\x12\x81\x01\n\x0e\x65mpty_strategy\x18\x02 \x01(\x0e\x32g.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies.EmptyCleanupStrategyH\x00\x12\x95\x01\n\x1cincremental_cleanup_strategy\x18\x03 \x01(\x0b\x32m.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies.IncrementalCleanupStrategyH\x00\x12\xa9\x01\n\'rocksdb_compact_filter_cleanup_strategy\x18\x04 \x01(\x0b\x32v.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies.RocksdbCompactFilterCleanupStrategyH\x00\x42\x11\n\x0f\x43leanupStrategy\"b\n\nStrategies\x12\x1c\n\x18\x46ULL_STATE_SCAN_SNAPSHOT\x10\x00\x12\x17\n\x13INCREMENTAL_CLEANUP\x10\x01\x12\x1d\n\x19ROCKSDB_COMPACTION_FILTER\x10\x02\"*\n\x14\x45mptyCleanupStrategy\x12\x12\n\x0e\x45MPTY_STRATEGY\x10\x00\"D\n\nUpdateType\x12\x0c\n\x08\x44isabled\x10\x00\x12\x14\n\x10OnCreateAndWrite\x10\x01\x12\x12\n\x0eOnReadAndWrite\x10\x02\"J\n\x0fStateVisibility\x12\x1f\n\x1bReturnExpiredIfNotCleanedUp\x10\x00\x12\x16\n\x12NeverReturnExpired\x10\x01\"+\n\x15TtlTimeCharacteristic\x12\x12\n\x0eProcessingTime\x10\x00\"\xf1\x07\n\x13\x43oderInfoDescriptor\x12`\n\x10\x66latten_row_type\x18\x01 \x01(\x0b\x32\x44.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.FlattenRowTypeH\x00\x12Q\n\x08row_type\x18\x02 \x01(\x0b\x32=.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.RowTypeH\x00\x12U\n\narrow_type\x18\x03 \x01(\x0b\x32?.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.ArrowTypeH\x00\x12k\n\x16over_window_arrow_type\x18\x04 \x01(\x0b\x32I.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.OverWindowArrowTypeH\x00\x12Q\n\x08raw_type\x18\x05 \x01(\x0b\x32=.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.RawTypeH\x00\x12H\n\x04mode\x18\x06 \x01(\x0e\x32:.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.Mode\x12\"\n\x1aseparated_with_end_message\x18\x07 \x01(\x08\x1aJ\n\x0e\x46lattenRowType\x12\x38\n\x06schema\x18\x01 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.Schema\x1a\x43\n\x07RowType\x12\x38\n\x06schema\x18\x01 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.Schema\x1a\x45\n\tArrowType\x12\x38\n\x06schema\x18\x01 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.Schema\x1aO\n\x13OverWindowArrowType\x12\x38\n\x06schema\x18\x01 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.Schema\x1aH\n\x07RawType\x12=\n\ttype_info\x18\x01 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\" \n\x04Mode\x12\n\n\x06SINGLE\x10\x00\x12\x0c\n\x08MULTIPLE\x10\x01\x42\x0b\n\tdata_typeB-\n\x1forg.apache.flink.fnexecution.v1B\nFlinkFnApib\x06proto3') + serialized_pb=_b('\n\x18\x66link-fn-execution.proto\x12 org.apache.flink.fn_execution.v1\"\x86\x01\n\x05Input\x12\x44\n\x03udf\x18\x01 \x01(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunctionH\x00\x12\x15\n\x0binputOffset\x18\x02 \x01(\x05H\x00\x12\x17\n\rinputConstant\x18\x03 \x01(\x0cH\x00\x42\x07\n\x05input\"\xa8\x01\n\x13UserDefinedFunction\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12\x37\n\x06inputs\x18\x02 \x03(\x0b\x32\'.org.apache.flink.fn_execution.v1.Input\x12\x14\n\x0cwindow_index\x18\x03 \x01(\x05\x12\x1a\n\x12takes_row_as_input\x18\x04 \x01(\x08\x12\x15\n\ris_pandas_udf\x18\x05 \x01(\x08\"\xcb\x01\n\x14UserDefinedFunctions\x12\x43\n\x04udfs\x18\x01 \x03(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunction\x12\x16\n\x0emetric_enabled\x18\x02 \x01(\x08\x12=\n\x07windows\x18\x03 \x03(\x0b\x32,.org.apache.flink.fn_execution.v1.OverWindow\x12\x17\n\x0fprofile_enabled\x18\x04 \x01(\x08\"\xdd\x02\n\nOverWindow\x12L\n\x0bwindow_type\x18\x01 \x01(\x0e\x32\x37.org.apache.flink.fn_execution.v1.OverWindow.WindowType\x12\x16\n\x0elower_boundary\x18\x02 \x01(\x03\x12\x16\n\x0eupper_boundary\x18\x03 \x01(\x03\"\xd0\x01\n\nWindowType\x12\x13\n\x0fRANGE_UNBOUNDED\x10\x00\x12\x1d\n\x19RANGE_UNBOUNDED_PRECEDING\x10\x01\x12\x1d\n\x19RANGE_UNBOUNDED_FOLLOWING\x10\x02\x12\x11\n\rRANGE_SLIDING\x10\x03\x12\x11\n\rROW_UNBOUNDED\x10\x04\x12\x1b\n\x17ROW_UNBOUNDED_PRECEDING\x10\x05\x12\x1b\n\x17ROW_UNBOUNDED_FOLLOWING\x10\x06\x12\x0f\n\x0bROW_SLIDING\x10\x07\"\x8b\x06\n\x1cUserDefinedAggregateFunction\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12\x37\n\x06inputs\x18\x02 \x03(\x0b\x32\'.org.apache.flink.fn_execution.v1.Input\x12Z\n\x05specs\x18\x03 \x03(\x0b\x32K.org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec\x12\x12\n\nfilter_arg\x18\x04 \x01(\x05\x12\x10\n\x08\x64istinct\x18\x05 \x01(\x08\x12\x1a\n\x12takes_row_as_input\x18\x06 \x01(\x08\x1a\x82\x04\n\x0c\x44\x61taViewSpec\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x13\n\x0b\x66ield_index\x18\x02 \x01(\x05\x12i\n\tlist_view\x18\x03 \x01(\x0b\x32T.org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.ListViewH\x00\x12g\n\x08map_view\x18\x04 \x01(\x0b\x32S.org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.MapViewH\x00\x1aT\n\x08ListView\x12H\n\x0c\x65lement_type\x18\x01 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x1a\x97\x01\n\x07MapView\x12\x44\n\x08key_type\x18\x01 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x12\x46\n\nvalue_type\x18\x02 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldTypeB\x0b\n\tdata_view\"\xac\x04\n\x0bGroupWindow\x12M\n\x0bwindow_type\x18\x01 \x01(\x0e\x32\x38.org.apache.flink.fn_execution.v1.GroupWindow.WindowType\x12\x16\n\x0eis_time_window\x18\x02 \x01(\x08\x12\x14\n\x0cwindow_slide\x18\x03 \x01(\x03\x12\x13\n\x0bwindow_size\x18\x04 \x01(\x03\x12\x12\n\nwindow_gap\x18\x05 \x01(\x03\x12\x13\n\x0bis_row_time\x18\x06 \x01(\x08\x12\x18\n\x10time_field_index\x18\x07 \x01(\x05\x12\x17\n\x0f\x61llowedLateness\x18\x08 \x01(\x03\x12U\n\x0fnamedProperties\x18\t \x03(\x0e\x32<.org.apache.flink.fn_execution.v1.GroupWindow.WindowProperty\x12\x16\n\x0eshift_timezone\x18\n \x01(\t\"[\n\nWindowType\x12\x19\n\x15TUMBLING_GROUP_WINDOW\x10\x00\x12\x18\n\x14SLIDING_GROUP_WINDOW\x10\x01\x12\x18\n\x14SESSION_GROUP_WINDOW\x10\x02\"c\n\x0eWindowProperty\x12\x10\n\x0cWINDOW_START\x10\x00\x12\x0e\n\nWINDOW_END\x10\x01\x12\x16\n\x12ROW_TIME_ATTRIBUTE\x10\x02\x12\x17\n\x13PROC_TIME_ATTRIBUTE\x10\x03\"\x96\x04\n\x1dUserDefinedAggregateFunctions\x12L\n\x04udfs\x18\x01 \x03(\x0b\x32>.org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction\x12\x16\n\x0emetric_enabled\x18\x02 \x01(\x08\x12\x10\n\x08grouping\x18\x03 \x03(\x05\x12\x1e\n\x16generate_update_before\x18\x04 \x01(\x08\x12\x44\n\x08key_type\x18\x05 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x12\x1b\n\x13index_of_count_star\x18\x06 \x01(\x05\x12\x1e\n\x16state_cleaning_enabled\x18\x07 \x01(\x08\x12\x18\n\x10state_cache_size\x18\x08 \x01(\x05\x12!\n\x19map_state_read_cache_size\x18\t \x01(\x05\x12\"\n\x1amap_state_write_cache_size\x18\n \x01(\x05\x12\x1b\n\x13\x63ount_star_inserted\x18\x0b \x01(\x08\x12\x43\n\x0cgroup_window\x18\x0c \x01(\x0b\x32-.org.apache.flink.fn_execution.v1.GroupWindow\x12\x17\n\x0fprofile_enabled\x18\r \x01(\x08\"\xec\x0f\n\x06Schema\x12>\n\x06\x66ields\x18\x01 \x03(\x0b\x32..org.apache.flink.fn_execution.v1.Schema.Field\x1a\x97\x01\n\x07MapInfo\x12\x44\n\x08key_type\x18\x01 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x12\x46\n\nvalue_type\x18\x02 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x1a\x1d\n\x08TimeInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a\"\n\rTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a,\n\x17LocalZonedTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a\'\n\x12ZonedTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a/\n\x0b\x44\x65\x63imalInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x12\r\n\x05scale\x18\x02 \x01(\x05\x1a\x1c\n\nBinaryInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1f\n\rVarBinaryInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1a\n\x08\x43harInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1d\n\x0bVarCharInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\xb0\x08\n\tFieldType\x12\x44\n\ttype_name\x18\x01 \x01(\x0e\x32\x31.org.apache.flink.fn_execution.v1.Schema.TypeName\x12\x10\n\x08nullable\x18\x02 \x01(\x08\x12U\n\x17\x63ollection_element_type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldTypeH\x00\x12\x44\n\x08map_info\x18\x04 \x01(\x0b\x32\x30.org.apache.flink.fn_execution.v1.Schema.MapInfoH\x00\x12>\n\nrow_schema\x18\x05 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.SchemaH\x00\x12L\n\x0c\x64\x65\x63imal_info\x18\x06 \x01(\x0b\x32\x34.org.apache.flink.fn_execution.v1.Schema.DecimalInfoH\x00\x12\x46\n\ttime_info\x18\x07 \x01(\x0b\x32\x31.org.apache.flink.fn_execution.v1.Schema.TimeInfoH\x00\x12P\n\x0etimestamp_info\x18\x08 \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.Schema.TimestampInfoH\x00\x12\x66\n\x1alocal_zoned_timestamp_info\x18\t \x01(\x0b\x32@.org.apache.flink.fn_execution.v1.Schema.LocalZonedTimestampInfoH\x00\x12[\n\x14zoned_timestamp_info\x18\n \x01(\x0b\x32;.org.apache.flink.fn_execution.v1.Schema.ZonedTimestampInfoH\x00\x12J\n\x0b\x62inary_info\x18\x0b \x01(\x0b\x32\x33.org.apache.flink.fn_execution.v1.Schema.BinaryInfoH\x00\x12Q\n\x0fvar_binary_info\x18\x0c \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.Schema.VarBinaryInfoH\x00\x12\x46\n\tchar_info\x18\r \x01(\x0b\x32\x31.org.apache.flink.fn_execution.v1.Schema.CharInfoH\x00\x12M\n\rvar_char_info\x18\x0e \x01(\x0b\x32\x34.org.apache.flink.fn_execution.v1.Schema.VarCharInfoH\x00\x42\x0b\n\ttype_info\x1al\n\x05\x46ield\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x02 \x01(\t\x12@\n\x04type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\"\xa1\x02\n\x08TypeName\x12\x07\n\x03ROW\x10\x00\x12\x0b\n\x07TINYINT\x10\x01\x12\x0c\n\x08SMALLINT\x10\x02\x12\x07\n\x03INT\x10\x03\x12\n\n\x06\x42IGINT\x10\x04\x12\x0b\n\x07\x44\x45\x43IMAL\x10\x05\x12\t\n\x05\x46LOAT\x10\x06\x12\n\n\x06\x44OUBLE\x10\x07\x12\x08\n\x04\x44\x41TE\x10\x08\x12\x08\n\x04TIME\x10\t\x12\r\n\tTIMESTAMP\x10\n\x12\x0b\n\x07\x42OOLEAN\x10\x0b\x12\n\n\x06\x42INARY\x10\x0c\x12\r\n\tVARBINARY\x10\r\x12\x08\n\x04\x43HAR\x10\x0e\x12\x0b\n\x07VARCHAR\x10\x0f\x12\x0f\n\x0b\x42\x41SIC_ARRAY\x10\x10\x12\x07\n\x03MAP\x10\x11\x12\x0c\n\x08MULTISET\x10\x12\x12\x19\n\x15LOCAL_ZONED_TIMESTAMP\x10\x13\x12\x13\n\x0fZONED_TIMESTAMP\x10\x14\"\xf7\x08\n\x08TypeInfo\x12\x46\n\ttype_name\x18\x01 \x01(\x0e\x32\x33.org.apache.flink.fn_execution.v1.TypeInfo.TypeName\x12M\n\x17\x63ollection_element_type\x18\x02 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfoH\x00\x12O\n\rrow_type_info\x18\x03 \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.TypeInfo.RowTypeInfoH\x00\x12S\n\x0ftuple_type_info\x18\x04 \x01(\x0b\x32\x38.org.apache.flink.fn_execution.v1.TypeInfo.TupleTypeInfoH\x00\x12O\n\rmap_type_info\x18\x05 \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.TypeInfo.MapTypeInfoH\x00\x1a\x8b\x01\n\x0bMapTypeInfo\x12<\n\x08key_type\x18\x01 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\x12>\n\nvalue_type\x18\x02 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\x1a\xb8\x01\n\x0bRowTypeInfo\x12L\n\x06\x66ields\x18\x01 \x03(\x0b\x32<.org.apache.flink.fn_execution.v1.TypeInfo.RowTypeInfo.Field\x1a[\n\x05\x46ield\x12\x12\n\nfield_name\x18\x01 \x01(\t\x12>\n\nfield_type\x18\x02 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\x1aP\n\rTupleTypeInfo\x12?\n\x0b\x66ield_types\x18\x01 \x03(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\"\xb4\x02\n\x08TypeName\x12\x07\n\x03ROW\x10\x00\x12\n\n\x06STRING\x10\x01\x12\x08\n\x04\x42YTE\x10\x02\x12\x0b\n\x07\x42OOLEAN\x10\x03\x12\t\n\x05SHORT\x10\x04\x12\x07\n\x03INT\x10\x05\x12\x08\n\x04LONG\x10\x06\x12\t\n\x05\x46LOAT\x10\x07\x12\n\n\x06\x44OUBLE\x10\x08\x12\x08\n\x04\x43HAR\x10\t\x12\x0b\n\x07\x42IG_INT\x10\n\x12\x0b\n\x07\x42IG_DEC\x10\x0b\x12\x0c\n\x08SQL_DATE\x10\x0c\x12\x0c\n\x08SQL_TIME\x10\r\x12\x11\n\rSQL_TIMESTAMP\x10\x0e\x12\x0f\n\x0b\x42\x41SIC_ARRAY\x10\x0f\x12\x13\n\x0fPRIMITIVE_ARRAY\x10\x10\x12\t\n\x05TUPLE\x10\x11\x12\x08\n\x04LIST\x10\x12\x12\x07\n\x03MAP\x10\x13\x12\x11\n\rPICKLED_BYTES\x10\x14\x12\x10\n\x0cOBJECT_ARRAY\x10\x15\x12\x0b\n\x07INSTANT\x10\x16\x42\x0b\n\ttype_info\"\xc7\x07\n\x1dUserDefinedDataStreamFunction\x12\x63\n\rfunction_type\x18\x01 \x01(\x0e\x32L.org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.FunctionType\x12g\n\x0fruntime_context\x18\x02 \x01(\x0b\x32N.org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext\x12\x0f\n\x07payload\x18\x03 \x01(\x0c\x12\x16\n\x0emetric_enabled\x18\x04 \x01(\x08\x12\x41\n\rkey_type_info\x18\x05 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\x12\x17\n\x0fprofile_enabled\x18\x06 \x01(\x08\x12\x18\n\x10state_cache_size\x18\x07 \x01(\x05\x12!\n\x19map_state_read_cache_size\x18\x08 \x01(\x05\x12\"\n\x1amap_state_write_cache_size\x18\t \x01(\x05\x1a*\n\x0cJobParameter\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t\x1a\xd0\x02\n\x0eRuntimeContext\x12\x11\n\ttask_name\x18\x01 \x01(\t\x12\x1f\n\x17task_name_with_subtasks\x18\x02 \x01(\t\x12#\n\x1bnumber_of_parallel_subtasks\x18\x03 \x01(\x05\x12\'\n\x1fmax_number_of_parallel_subtasks\x18\x04 \x01(\x05\x12\x1d\n\x15index_of_this_subtask\x18\x05 \x01(\x05\x12\x16\n\x0e\x61ttempt_number\x18\x06 \x01(\x05\x12\x64\n\x0ejob_parameters\x18\x07 \x03(\x0b\x32L.org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.JobParameter\x12\x1f\n\x17in_batch_execution_mode\x18\x08 \x01(\x08\"s\n\x0c\x46unctionType\x12\x0b\n\x07PROCESS\x10\x00\x12\x0e\n\nCO_PROCESS\x10\x01\x12\x11\n\rKEYED_PROCESS\x10\x02\x12\x14\n\x10KEYED_CO_PROCESS\x10\x03\x12\n\n\x06WINDOW\x10\x04\x12\x11\n\rREVISE_OUTPUT\x10\x64\"\xe4\x0e\n\x0fStateDescriptor\x12\x12\n\nstate_name\x18\x01 \x01(\t\x12Z\n\x10state_ttl_config\x18\x02 \x01(\x0b\x32@.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig\x1a\xe0\r\n\x0eStateTTLConfig\x12`\n\x0bupdate_type\x18\x01 \x01(\x0e\x32K.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.UpdateType\x12j\n\x10state_visibility\x18\x02 \x01(\x0e\x32P.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.StateVisibility\x12w\n\x17ttl_time_characteristic\x18\x03 \x01(\x0e\x32V.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.TtlTimeCharacteristic\x12\x0b\n\x03ttl\x18\x04 \x01(\x03\x12n\n\x12\x63leanup_strategies\x18\x05 \x01(\x0b\x32R.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies\x1a\xca\x08\n\x11\x43leanupStrategies\x12 \n\x18is_cleanup_in_background\x18\x01 \x01(\x08\x12y\n\nstrategies\x18\x02 \x03(\x0b\x32\x65.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies.MapStrategiesEntry\x1aX\n\x1aIncrementalCleanupStrategy\x12\x14\n\x0c\x63leanup_size\x18\x01 \x01(\x05\x12$\n\x1crun_cleanup_for_every_record\x18\x02 \x01(\x08\x1aK\n#RocksdbCompactFilterCleanupStrategy\x12$\n\x1cquery_time_after_num_entries\x18\x01 \x01(\x03\x1a\xe0\x04\n\x12MapStrategiesEntry\x12o\n\x08strategy\x18\x01 \x01(\x0e\x32].org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies.Strategies\x12\x81\x01\n\x0e\x65mpty_strategy\x18\x02 \x01(\x0e\x32g.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies.EmptyCleanupStrategyH\x00\x12\x95\x01\n\x1cincremental_cleanup_strategy\x18\x03 \x01(\x0b\x32m.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies.IncrementalCleanupStrategyH\x00\x12\xa9\x01\n\'rocksdb_compact_filter_cleanup_strategy\x18\x04 \x01(\x0b\x32v.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies.RocksdbCompactFilterCleanupStrategyH\x00\x42\x11\n\x0f\x43leanupStrategy\"b\n\nStrategies\x12\x1c\n\x18\x46ULL_STATE_SCAN_SNAPSHOT\x10\x00\x12\x17\n\x13INCREMENTAL_CLEANUP\x10\x01\x12\x1d\n\x19ROCKSDB_COMPACTION_FILTER\x10\x02\"*\n\x14\x45mptyCleanupStrategy\x12\x12\n\x0e\x45MPTY_STRATEGY\x10\x00\"D\n\nUpdateType\x12\x0c\n\x08\x44isabled\x10\x00\x12\x14\n\x10OnCreateAndWrite\x10\x01\x12\x12\n\x0eOnReadAndWrite\x10\x02\"J\n\x0fStateVisibility\x12\x1f\n\x1bReturnExpiredIfNotCleanedUp\x10\x00\x12\x16\n\x12NeverReturnExpired\x10\x01\"+\n\x15TtlTimeCharacteristic\x12\x12\n\x0eProcessingTime\x10\x00\"\xf1\x07\n\x13\x43oderInfoDescriptor\x12`\n\x10\x66latten_row_type\x18\x01 \x01(\x0b\x32\x44.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.FlattenRowTypeH\x00\x12Q\n\x08row_type\x18\x02 \x01(\x0b\x32=.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.RowTypeH\x00\x12U\n\narrow_type\x18\x03 \x01(\x0b\x32?.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.ArrowTypeH\x00\x12k\n\x16over_window_arrow_type\x18\x04 \x01(\x0b\x32I.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.OverWindowArrowTypeH\x00\x12Q\n\x08raw_type\x18\x05 \x01(\x0b\x32=.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.RawTypeH\x00\x12H\n\x04mode\x18\x06 \x01(\x0e\x32:.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.Mode\x12\"\n\x1aseparated_with_end_message\x18\x07 \x01(\x08\x1aJ\n\x0e\x46lattenRowType\x12\x38\n\x06schema\x18\x01 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.Schema\x1a\x43\n\x07RowType\x12\x38\n\x06schema\x18\x01 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.Schema\x1a\x45\n\tArrowType\x12\x38\n\x06schema\x18\x01 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.Schema\x1aO\n\x13OverWindowArrowType\x12\x38\n\x06schema\x18\x01 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.Schema\x1aH\n\x07RawType\x12=\n\ttype_info\x18\x01 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\" \n\x04Mode\x12\n\n\x06SINGLE\x10\x00\x12\x0c\n\x08MULTIPLE\x10\x01\x42\x0b\n\tdata_typeB-\n\x1forg.apache.flink.fnexecution.v1B\nFlinkFnApib\x06proto3') ) @@ -380,8 +380,8 @@ ], containing_type=None, options=None, - serialized_start=6739, - serialized_end=6854, + serialized_start=6836, + serialized_end=6951, ) _sym_db.RegisterEnumDescriptor(_USERDEFINEDDATASTREAMFUNCTION_FUNCTIONTYPE) @@ -406,8 +406,8 @@ ], containing_type=None, options=None, - serialized_start=8416, - serialized_end=8514, + serialized_start=8513, + serialized_end=8611, ) _sym_db.RegisterEnumDescriptor(_STATEDESCRIPTOR_STATETTLCONFIG_CLEANUPSTRATEGIES_STRATEGIES) @@ -424,8 +424,8 @@ ], containing_type=None, options=None, - serialized_start=8516, - serialized_end=8558, + serialized_start=8613, + serialized_end=8655, ) _sym_db.RegisterEnumDescriptor(_STATEDESCRIPTOR_STATETTLCONFIG_CLEANUPSTRATEGIES_EMPTYCLEANUPSTRATEGY) @@ -450,8 +450,8 @@ ], containing_type=None, options=None, - serialized_start=8560, - serialized_end=8628, + serialized_start=8657, + serialized_end=8725, ) _sym_db.RegisterEnumDescriptor(_STATEDESCRIPTOR_STATETTLCONFIG_UPDATETYPE) @@ -472,8 +472,8 @@ ], containing_type=None, options=None, - serialized_start=8630, - serialized_end=8704, + serialized_start=8727, + serialized_end=8801, ) _sym_db.RegisterEnumDescriptor(_STATEDESCRIPTOR_STATETTLCONFIG_STATEVISIBILITY) @@ -490,8 +490,8 @@ ], containing_type=None, options=None, - serialized_start=8706, - serialized_end=8749, + serialized_start=8803, + serialized_end=8846, ) _sym_db.RegisterEnumDescriptor(_STATEDESCRIPTOR_STATETTLCONFIG_TTLTIMECHARACTERISTIC) @@ -512,8 +512,8 @@ ], containing_type=None, options=None, - serialized_start=9716, - serialized_end=9748, + serialized_start=9813, + serialized_end=9845, ) _sym_db.RegisterEnumDescriptor(_CODERINFODESCRIPTOR_MODE) @@ -1865,8 +1865,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=6356, - serialized_end=6398, + serialized_start=6453, + serialized_end=6495, ) _USERDEFINEDDATASTREAMFUNCTION_RUNTIMECONTEXT = _descriptor.Descriptor( @@ -1944,8 +1944,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=6401, - serialized_end=6737, + serialized_start=6498, + serialized_end=6834, ) _USERDEFINEDDATASTREAMFUNCTION = _descriptor.Descriptor( @@ -1997,6 +1997,27 @@ message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='state_cache_size', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.state_cache_size', index=6, + number=7, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='map_state_read_cache_size', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.map_state_read_cache_size', index=7, + number=8, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='map_state_write_cache_size', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.map_state_write_cache_size', index=8, + number=9, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), ], extensions=[ ], @@ -2011,7 +2032,7 @@ oneofs=[ ], serialized_start=5984, - serialized_end=6854, + serialized_end=6951, ) @@ -2048,8 +2069,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=7638, - serialized_end=7726, + serialized_start=7735, + serialized_end=7823, ) _STATEDESCRIPTOR_STATETTLCONFIG_CLEANUPSTRATEGIES_ROCKSDBCOMPACTFILTERCLEANUPSTRATEGY = _descriptor.Descriptor( @@ -2078,8 +2099,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=7728, - serialized_end=7803, + serialized_start=7825, + serialized_end=7900, ) _STATEDESCRIPTOR_STATETTLCONFIG_CLEANUPSTRATEGIES_MAPSTRATEGIESENTRY = _descriptor.Descriptor( @@ -2132,8 +2153,8 @@ name='CleanupStrategy', full_name='org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies.MapStrategiesEntry.CleanupStrategy', index=0, containing_type=None, fields=[]), ], - serialized_start=7806, - serialized_end=8414, + serialized_start=7903, + serialized_end=8511, ) _STATEDESCRIPTOR_STATETTLCONFIG_CLEANUPSTRATEGIES = _descriptor.Descriptor( @@ -2171,8 +2192,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=7460, - serialized_end=8558, + serialized_start=7557, + serialized_end=8655, ) _STATEDESCRIPTOR_STATETTLCONFIG = _descriptor.Descriptor( @@ -2232,8 +2253,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=6989, - serialized_end=8749, + serialized_start=7086, + serialized_end=8846, ) _STATEDESCRIPTOR = _descriptor.Descriptor( @@ -2269,8 +2290,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=6857, - serialized_end=8749, + serialized_start=6954, + serialized_end=8846, ) @@ -2300,8 +2321,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=9345, - serialized_end=9419, + serialized_start=9442, + serialized_end=9516, ) _CODERINFODESCRIPTOR_ROWTYPE = _descriptor.Descriptor( @@ -2330,8 +2351,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=9421, - serialized_end=9488, + serialized_start=9518, + serialized_end=9585, ) _CODERINFODESCRIPTOR_ARROWTYPE = _descriptor.Descriptor( @@ -2360,8 +2381,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=9490, - serialized_end=9559, + serialized_start=9587, + serialized_end=9656, ) _CODERINFODESCRIPTOR_OVERWINDOWARROWTYPE = _descriptor.Descriptor( @@ -2390,8 +2411,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=9561, - serialized_end=9640, + serialized_start=9658, + serialized_end=9737, ) _CODERINFODESCRIPTOR_RAWTYPE = _descriptor.Descriptor( @@ -2420,8 +2441,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=9642, - serialized_end=9714, + serialized_start=9739, + serialized_end=9811, ) _CODERINFODESCRIPTOR = _descriptor.Descriptor( @@ -2496,8 +2517,8 @@ name='data_type', full_name='org.apache.flink.fn_execution.v1.CoderInfoDescriptor.data_type', index=0, containing_type=None, fields=[]), ], - serialized_start=8752, - serialized_end=9761, + serialized_start=8849, + serialized_end=9858, ) _INPUT.fields_by_name['udf'].message_type = _USERDEFINEDFUNCTION diff --git a/flink-python/pyflink/fn_execution/state_impl.py b/flink-python/pyflink/fn_execution/state_impl.py index bccb5a6ea8971..38668dd4e83c1 100644 --- a/flink-python/pyflink/fn_execution/state_impl.py +++ b/flink-python/pyflink/fn_execution/state_impl.py @@ -87,6 +87,9 @@ def __len__(self): def __iter__(self): return iter(self._cache.values()) + def __contains__(self, key): + return key in self._cache + class SynchronousKvRuntimeState(InternalKvState, ABC): """ @@ -146,7 +149,8 @@ def get_internal_state(self): return self._internal_state def _maybe_clear_write_cache(self): - if self._cache_type == SynchronousKvRuntimeState.CacheType.DISABLE_CACHE: + if self._cache_type == SynchronousKvRuntimeState.CacheType.DISABLE_CACHE or \ + self._remote_state_backend._state_cache_size <= 0: self._internal_state.commit() self._internal_state._cleared = False self._internal_state._added_elements = [] @@ -1155,13 +1159,14 @@ def clear_cached_iterators(self): self._map_state_handler.clear(self._clear_iterator_mark) def merge_namespaces(self, state: SynchronousMergingRuntimeState, target, sources, ttl_config): + for source in sources: + state.set_current_namespace(source) + self.commit_internal_state(state.get_internal_state()) state.set_current_namespace(target) self.commit_internal_state(state.get_internal_state()) encoded_target_namespace = self._encode_namespace(target) - encoded_namespaces = [encoded_target_namespace] - for source in sources: - encoded_namespaces.append(self._encode_namespace(source)) - self.clear_state_cache(state, encoded_namespaces) + encoded_namespaces = [self._encode_namespace(source) for source in sources] + self.clear_state_cache(state, [encoded_target_namespace] + encoded_namespaces) state_key = self.get_bag_state_key( state.name, self._encoded_current_key, encoded_target_namespace, ttl_config) diff --git a/flink-python/pyflink/fn_execution/table/window_aggregate_fast.pyx b/flink-python/pyflink/fn_execution/table/window_aggregate_fast.pyx index 7a21249a8542f..bcbee00f0a716 100644 --- a/flink-python/pyflink/fn_execution/table/window_aggregate_fast.pyx +++ b/flink-python/pyflink/fn_execution/table/window_aggregate_fast.pyx @@ -25,11 +25,11 @@ from pyflink.fn_execution.table.aggregate_fast cimport DistinctViewDescriptor, R from pyflink.fn_execution.coder_impl_fast cimport InternalRowKind import datetime -import sys from typing import List, Dict import pytz +from pyflink.common.constants import MAX_LONG_VALUE from pyflink.fn_execution.datastream.timerservice_impl import LegacyInternalTimerServiceImpl from pyflink.fn_execution.coders import PickleCoder from pyflink.fn_execution.table.state_data_view import DataViewSpec, ListViewSpec, MapViewSpec, \ @@ -42,8 +42,6 @@ from pyflink.fn_execution.table.window_process_function import GeneralWindowProc from pyflink.fn_execution.table.window_trigger import Trigger from pyflink.table.udf import ImperativeAggregateFunction -MAX_LONG_VALUE = sys.maxsize - cdef InternalRow join_row(list left, list right, InternalRowKind row_kind): return InternalRow(left.__add__(right), row_kind) diff --git a/flink-python/pyflink/fn_execution/table/window_aggregate_slow.py b/flink-python/pyflink/fn_execution/table/window_aggregate_slow.py index a7e557b385a18..9fc1f2e6f3814 100644 --- a/flink-python/pyflink/fn_execution/table/window_aggregate_slow.py +++ b/flink-python/pyflink/fn_execution/table/window_aggregate_slow.py @@ -16,13 +16,13 @@ # limitations under the License. ################################################################################ import datetime -import sys from abc import ABC, abstractmethod from typing import TypeVar, Generic, List, Dict import pytz from pyflink.common import Row, RowKind +from pyflink.common.constants import MAX_LONG_VALUE from pyflink.fn_execution.datastream.timerservice import InternalTimer from pyflink.fn_execution.datastream.timerservice_impl import LegacyInternalTimerServiceImpl from pyflink.fn_execution.coders import PickleCoder @@ -37,7 +37,6 @@ from pyflink.fn_execution.table.window_trigger import Trigger from pyflink.table.udf import ImperativeAggregateFunction, FunctionContext -MAX_LONG_VALUE = sys.maxsize N = TypeVar('N') diff --git a/flink-python/pyflink/fn_execution/table/window_context.py b/flink-python/pyflink/fn_execution/table/window_context.py index 2e1946e0d6913..5e2bebc86e98a 100644 --- a/flink-python/pyflink/fn_execution/table/window_context.py +++ b/flink-python/pyflink/fn_execution/table/window_context.py @@ -15,10 +15,10 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ -import sys from abc import ABC, abstractmethod from typing import Generic, TypeVar, List, Iterable +from pyflink.common.constants import MAX_LONG_VALUE from pyflink.datastream.state import StateDescriptor, State, ValueStateDescriptor, \ ListStateDescriptor, MapStateDescriptor from pyflink.datastream.window import TimeWindow, CountWindow @@ -26,8 +26,6 @@ from pyflink.fn_execution.coders import from_type_info, MapCoder, GenericArrayCoder from pyflink.fn_execution.internal_state import InternalMergingState -MAX_LONG_VALUE = sys.maxsize - K = TypeVar('K') W = TypeVar('W', TimeWindow, CountWindow) diff --git a/flink-python/pyflink/fn_execution/table/window_process_function.py b/flink-python/pyflink/fn_execution/table/window_process_function.py index 733d4c1fdac2e..333d903f90182 100644 --- a/flink-python/pyflink/fn_execution/table/window_process_function.py +++ b/flink-python/pyflink/fn_execution/table/window_process_function.py @@ -15,18 +15,16 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ -import sys from abc import abstractmethod, ABC from typing import Generic, List, Iterable, Dict, Set from pyflink.common import Row +from pyflink.common.constants import MAX_LONG_VALUE from pyflink.datastream.state import MapState from pyflink.fn_execution.table.window_assigner import WindowAssigner, PanedWindowAssigner, \ MergingWindowAssigner from pyflink.fn_execution.table.window_context import Context, K, W -MAX_LONG_VALUE = sys.maxsize - def join_row(left: List, right: List): return Row(*(left + right)) diff --git a/flink-python/pyflink/java_gateway.py b/flink-python/pyflink/java_gateway.py index b1272407ac481..fdb47a8c754ec 100644 --- a/flink-python/pyflink/java_gateway.py +++ b/flink-python/pyflink/java_gateway.py @@ -132,6 +132,7 @@ def import_flink_view(gateway): """ # Import the classes used by PyFlink java_import(gateway.jvm, "org.apache.flink.table.api.*") + java_import(gateway.jvm, "org.apache.flink.table.api.config.*") java_import(gateway.jvm, "org.apache.flink.table.api.java.*") java_import(gateway.jvm, "org.apache.flink.table.api.bridge.java.*") java_import(gateway.jvm, "org.apache.flink.table.api.dataview.*") diff --git a/flink-python/pyflink/proto/flink-fn-execution.proto b/flink-python/pyflink/proto/flink-fn-execution.proto index 95ee2f5c0377d..7ea69a55f8b93 100644 --- a/flink-python/pyflink/proto/flink-fn-execution.proto +++ b/flink-python/pyflink/proto/flink-fn-execution.proto @@ -371,6 +371,13 @@ message UserDefinedDataStreamFunction { bool metric_enabled = 4; TypeInfo key_type_info = 5; bool profile_enabled = 6; + + // The state cache size. + int32 state_cache_size = 7; + // The map state read cache size. + int32 map_state_read_cache_size = 8; + // The map_state_write_cache_size. + int32 map_state_write_cache_size = 9; } // A representation of State diff --git a/flink-python/pyflink/table/environment_settings.py b/flink-python/pyflink/table/environment_settings.py index 578567d7094cf..bc615824961b9 100644 --- a/flink-python/pyflink/table/environment_settings.py +++ b/flink-python/pyflink/table/environment_settings.py @@ -15,6 +15,8 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ +import warnings + from pyflink.java_gateway import get_gateway from pyflink.common import Configuration @@ -37,19 +39,28 @@ class EnvironmentSettings(object): ... .with_built_in_database_name("my_database") \\ ... .build() - :func:`EnvironmentSettings.in_streaming_mode` or :func:`EnvironmentSettings.in_batch_mode` + :func:`~EnvironmentSettings.in_streaming_mode` or :func:`~EnvironmentSettings.in_batch_mode` might be convenient as shortcuts. """ class Builder(object): """ - A builder for :class:`EnvironmentSettings`. + A builder for :class:`~EnvironmentSettings`. """ def __init__(self): gateway = get_gateway() self._j_builder = gateway.jvm.EnvironmentSettings.Builder() + def with_configuration(self, config: Configuration) -> 'EnvironmentSettings.Builder': + """ + Creates the EnvironmentSetting with specified Configuration. + + :return: EnvironmentSettings. + """ + self._j_builder = self._j_builder.withConfiguration(config._j_configuration) + return self + def in_batch_mode(self) -> 'EnvironmentSettings.Builder': """ Sets that the components should work in a batch mode. Streaming mode by default. @@ -155,9 +166,21 @@ def to_configuration(self) -> Configuration: Convert to `pyflink.common.Configuration`. :return: Configuration with specified value. + + .. note:: Deprecated in 1.15. Please use + :func:`EnvironmentSettings.get_configuration` instead. """ + warnings.warn("Deprecated in 1.15.", DeprecationWarning) return Configuration(j_configuration=self._j_environment_settings.toConfiguration()) + def get_configuration(self) -> Configuration: + """ + Get the underlying `pyflink.common.Configuration`. + + :return: Configuration with specified value. + """ + return Configuration(j_configuration=self._j_environment_settings.getConfiguration()) + @staticmethod def new_instance() -> 'EnvironmentSettings.Builder': """ @@ -173,7 +196,11 @@ def from_configuration(config: Configuration) -> 'EnvironmentSettings': Creates the EnvironmentSetting with specified Configuration. :return: EnvironmentSettings. + + .. note:: Deprecated in 1.15. Please use + :func:`EnvironmentSettings.Builder.with_configuration` instead. """ + warnings.warn("Deprecated in 1.15.", DeprecationWarning) return EnvironmentSettings( get_gateway().jvm.EnvironmentSettings.fromConfiguration(config._j_configuration)) diff --git a/flink-python/pyflink/table/expressions.py b/flink-python/pyflink/table/expressions.py index 2467db3645cdc..d870f825e20da 100644 --- a/flink-python/pyflink/table/expressions.py +++ b/flink-python/pyflink/table/expressions.py @@ -438,7 +438,7 @@ def rand(seed: Union[int, Expression[int]] = None) -> Expression[float]: def rand_integer(bound: Union[int, Expression[int]], seed: Union[int, Expression[int]] = None) -> Expression: """ - Returns a pseudorandom integer value between 0.0 (inclusive) and the specified value + Returns a pseudorandom integer value between 0 (inclusive) and the specified value (exclusive) with a initial seed if specified. Two rand_integer() functions will return identical sequences of numbers if they have same initial seed and same bound. """ diff --git a/flink-python/pyflink/table/functions.py b/flink-python/pyflink/table/functions.py index d5b78cee0c318..9fe27dfa1d864 100644 --- a/flink-python/pyflink/table/functions.py +++ b/flink-python/pyflink/table/functions.py @@ -15,16 +15,13 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ -import sys import time from abc import abstractmethod from decimal import Decimal +from pyflink.common.constants import MAX_LONG_VALUE, MIN_LONG_VALUE from pyflink.table import AggregateFunction, MapView, ListView -MAX_LONG_VALUE = sys.maxsize -MIN_LONG_VALUE = -MAX_LONG_VALUE - 1 - class AvgAggFunction(AggregateFunction): diff --git a/flink-python/pyflink/table/statement_set.py b/flink-python/pyflink/table/statement_set.py index c87fbea375c6a..4c3bb7a4625d7 100644 --- a/flink-python/pyflink/table/statement_set.py +++ b/flink-python/pyflink/table/statement_set.py @@ -27,14 +27,10 @@ class StatementSet(object): """ - A StatementSet accepts DML statements or Tables, - the planner can optimize all added statements and Tables together - and then submit as one job. + A :class:`~StatementSet` accepts pipelines defined by DML statements or :class:`~Table` objects. + The planner can optimize all added statements together and then submit them as one job. - .. note:: - - The added statements and Tables will be cleared - when calling the `execute` method. + The added statements will be cleared when calling the :func:`~StatementSet.execute` method. .. versionadded:: 1.11.0 """ @@ -93,9 +89,9 @@ def add_insert(self, >>> stmt_set = table_env.create_statement_set() >>> source_table = table_env.from_path("SourceTable") - >>> sink_descriptor = TableDescriptor.for_connector("blackhole") + >>> sink_descriptor = TableDescriptor.for_connector("blackhole") \\ ... .schema(Schema.new_builder() - ... .build()) + ... .build()) \\ ... .build() >>> stmt_set.add_insert(sink_descriptor, source_table) diff --git a/flink-python/pyflink/table/table.py b/flink-python/pyflink/table/table.py index 484d7583a72f6..200cafa23424d 100644 --- a/flink-python/pyflink/table/table.py +++ b/flink-python/pyflink/table/table.py @@ -40,32 +40,56 @@ class Table(object): - """ - A :class:`~pyflink.table.Table` is the core component of the Table API. + A :class:`~pyflink.table.Table` object is the core abstraction of the Table API. Similar to how the DataStream API has DataStream, the Table API is built around :class:`~pyflink.table.Table`. - Use the methods of :class:`~pyflink.table.Table` to transform data. + A :class:`~pyflink.table.Table` object describes a pipeline of data transformations. It does not + contain the data itself in any way. Instead, it describes how to read data from a table source, + and how to eventually write data to a table sink. The declared pipeline can be + printed, optimized, and eventually executed in a cluster. The pipeline can work with bounded or + unbounded streams which enables both streaming and batch scenarios. + + By the definition above, a :class:`~pyflink.table.Table` object can actually be considered as + a view in SQL terms. + + The initial :class:`~pyflink.table.Table` object is constructed by a + :class:`~pyflink.table.TableEnvironment`. For example, + :func:`~pyflink.table.TableEnvironment.from_path` obtains a table from a catalog. + Every :class:`~pyflink.table.Table` object has a schema that is available through + :func:`~pyflink.table.Table.get_schema`. A :class:`~pyflink.table.Table` object is + always associated with its original table environment during programming. + + Every transformation (i.e. :func:`~pyflink.table.Table.select`} or + :func:`~pyflink.table.Table.filter` on a :class:`~pyflink.table.Table` object leads to a new + :class:`~pyflink.table.Table` object. + + Use :func:`~pyflink.table.Table.execute` to execute the pipeline and retrieve the transformed + data locally during development. Otherwise, use :func:`~pyflink.table.Table.execute_insert` to + write the data into a table sink. + + Many methods of this class take one or more :class:`~pyflink.table.Expression` as parameters. + For fluent definition of expressions and easier readability, we recommend to add a star import: + + Example: + :: + + >>> from pyflink.table.expressions import * + + Check the documentation for more programming language specific APIs. + + The following example shows how to work with a :class:`~pyflink.table.Table` object. Example: :: - >>> env = StreamExecutionEnvironment.get_execution_environment() - >>> env.set_parallelism(1) - >>> t_env = StreamTableEnvironment.create(env) - >>> ... - >>> t_env.register_table_source("source", ...) - >>> t = t_env.from_path("source") - >>> t.select(...) - >>> ... - >>> t_env.register_table_sink("result", ...) - >>> t.execute_insert("result") - - Operations such as :func:`~pyflink.table.Table.join`, :func:`~pyflink.table.Table.select`, - :func:`~pyflink.table.Table.where` and :func:`~pyflink.table.Table.group_by` - take arguments in an expression string. Please refer to the documentation for - the expression syntax. + >>> from pyflink.table import TableEnvironment + >>> from pyflink.table.expressions import * + >>> env_settings = EnvironmentSettings.in_streaming_mode() + >>> t_env = TableEnvironment.create(env_settings) + >>> table = t_env.from_path("my_table").select(col("colA").trim(), col("colB") + 12) + >>> table.execute().print() """ def __init__(self, j_table, t_env): @@ -969,8 +993,8 @@ def to_pandas(self): """ self._t_env._before_execute() gateway = get_gateway() - max_arrow_batch_size = self._j_table.getTableEnvironment().getConfig().getConfiguration()\ - .getInteger(gateway.jvm.org.apache.flink.python.PythonOptions.MAX_ARROW_BATCH_SIZE) + max_arrow_batch_size = self._j_table.getTableEnvironment().getConfig()\ + .get(gateway.jvm.org.apache.flink.python.PythonOptions.MAX_ARROW_BATCH_SIZE) batches_iterator = gateway.jvm.org.apache.flink.table.runtime.arrow.ArrowUtils\ .collectAsPandasDataFrame(self._j_table, max_arrow_batch_size) if batches_iterator.hasNext(): diff --git a/flink-python/pyflink/table/table_config.py b/flink-python/pyflink/table/table_config.py index eacd6359e5259..5fa9896a56635 100644 --- a/flink-python/pyflink/table/table_config.py +++ b/flink-python/pyflink/table/table_config.py @@ -16,6 +16,7 @@ # limitations under the License. ################################################################################ import datetime +import warnings from py4j.compat import long from typing import Tuple @@ -26,18 +27,48 @@ __all__ = ['TableConfig'] +from pyflink.util.java_utils import add_jars_to_context_class_loader + class TableConfig(object): """ Configuration for the current :class:`TableEnvironment` session to adjust Table & SQL API programs. + This class is a pure API class that abstracts configuration from various sources. Currently, + configuration can be set in any of the following layers (in the given order): + + - flink-conf.yaml + - CLI parameters + - :class:`~pyflink.datastream.StreamExecutionEnvironment` when bridging to DataStream API + - :func:`~EnvironmentSettings.Builder.with_configuration` + - :func:`~TableConfig.set` + + The latter two represent the application-specific part of the configuration. They initialize + and directly modify :func:`~TableConfig.get_configuration`. Other layers represent the + configuration of the execution context and are immutable. + + The getter :func:`~TableConfig.get` gives read-only access to the full configuration. However, + application-specific configuration has precedence. Configuration of outer layers is used for + defaults and fallbacks. The setter :func:`~TableConfig.set` will only affect + application-specific configuration. + For common or important configuration options, this class provides getters and setters methods with detailed inline documentation. For more advanced configuration, users can directly access the underlying key-value map via :func:`~pyflink.table.TableConfig.get_configuration`. + Example: + :: + + >>> table_config = t_env.get_config() + >>> config = Configuration() + >>> config.set_string("parallelism.default", "128") \\ + ... .set_string("pipeline.auto-watermark-interval", "800ms") \\ + ... .set_string("execution.checkpointing.interval", "30s") + >>> table_config.add_configuration(config) + .. note:: Because options are read at different point in time when performing operations, it is @@ -47,10 +78,24 @@ class TableConfig(object): def __init__(self, j_table_config=None): gateway = get_gateway() if j_table_config is None: - self._j_table_config = gateway.jvm.TableConfig() + self._j_table_config = gateway.jvm.TableConfig.getDefault() else: self._j_table_config = j_table_config + def get(self, key: str, default_value: str) -> str: + """ + Returns the value associated with the given key as a string. + + :param key: The key pointing to the associated value. + :param default_value: The default value which is returned in case there is no value + associated with the given key. + :return: The (default) value associated with the given key. + """ + if self.get_configuration().contains_key(key): + return self.get_configuration().get_string(key, default_value) + else: + return self._j_table_config.getRootConfiguration().getString(key, default_value) + def set(self, key: str, value: str) -> 'TableConfig': """ Sets a string-based value for the given string-based key. @@ -58,6 +103,13 @@ def set(self, key: str, value: str) -> 'TableConfig': The value will be parsed by the framework on access. """ self._j_table_config.set(key, value) + + jvm = get_gateway().jvm + jars_key = jvm.org.apache.flink.configuration.PipelineOptions.JARS.key() + classpaths_key = jvm.org.apache.flink.configuration.PipelineOptions.CLASSPATHS.key() + if key in [jars_key, classpaths_key]: + add_jars_to_context_class_loader(value.split(";")) + return self def get_local_timezone(self) -> str: @@ -85,13 +137,21 @@ def set_local_timezone(self, timezone_id: str): def get_null_check(self) -> bool: """ A boolean value, "True" enables NULL check and "False" disables NULL check. + + .. note:: Deprecated in 1.15 and will be removed in next release. """ + warnings.warn("Deprecated in 1.15.", DeprecationWarning) + return self._j_table_config.getNullCheck() def set_null_check(self, null_check: bool): """ Sets the NULL check. If enabled, all fields need to be checked for NULL first. + + .. note:: Deprecated in 1.15 and will be removed in next release. """ + warnings.warn("Deprecated in 1.15.", DeprecationWarning) + if null_check is not None and isinstance(null_check, bool): self._j_table_config.setNullCheck(null_check) else: @@ -273,7 +333,11 @@ def set_decimal_context(self, precision: int, rounding_mode: str): :param precision: The precision of the decimal context. :param rounding_mode: The rounding mode of the decimal context. + + .. note:: Deprecated in 1.15 and will be removed in next release. """ + warnings.warn("Deprecated in 1.15.", DeprecationWarning) + if rounding_mode not in ( "UP", "DOWN", @@ -297,7 +361,11 @@ def get_decimal_context(self) -> Tuple[int, str]: .. seealso:: :func:`set_decimal_context` :return: the current context for decimal division calculation. + + .. note:: Deprecated in 1.15 and will be removed in next release. """ + warnings.warn("Deprecated in 1.15.", DeprecationWarning) + j_math_context = self._j_table_config.getDecimalContext() precision = j_math_context.getPrecision() rounding_mode = j_math_context.getRoundingMode().name() @@ -372,7 +440,7 @@ def set_python_executable(self, python_exec: str): .. versionadded:: 1.10.0 """ jvm = get_gateway().jvm - self.get_configuration().set_string(jvm.PythonOptions.PYTHON_EXECUTABLE.key(), python_exec) + self.set(jvm.PythonOptions.PYTHON_EXECUTABLE.key(), python_exec) def get_python_executable(self) -> str: """ diff --git a/flink-python/pyflink/table/table_environment.py b/flink-python/pyflink/table/table_environment.py index 57e10eadc0029..2e38ad9006a4d 100644 --- a/flink-python/pyflink/table/table_environment.py +++ b/flink-python/pyflink/table/table_environment.py @@ -1156,27 +1156,32 @@ def create_temporary_view(self, """ if isinstance(table_or_data_stream, Table): self._j_tenv.createTemporaryView(view_path, table_or_data_stream._j_table) - elif len(fields_or_schema) == 0: - self._j_tenv.createTemporaryView(view_path, table_or_data_stream._j_data_stream) - elif len(fields_or_schema) == 1 and isinstance(fields_or_schema[0], str): - self._j_tenv.createTemporaryView( - view_path, - table_or_data_stream._j_data_stream, - fields_or_schema[0]) - elif len(fields_or_schema) == 1 and isinstance(fields_or_schema[0], Schema): - self._j_tenv.createTemporaryView( - view_path, - table_or_data_stream._j_data_stream, - fields_or_schema[0]._j_schema) - elif (len(fields_or_schema) > 0 and - all(isinstance(elem, Expression) for elem in fields_or_schema)): - self._j_tenv.createTemporaryView( - view_path, - table_or_data_stream._j_data_stream, - to_expression_jarray(fields_or_schema)) else: - raise ValueError("Invalid arguments for 'fields': %r" % - ','.join([repr(item) for item in fields_or_schema])) + j_data_stream = table_or_data_stream._j_data_stream + JPythonConfigUtil = get_gateway().jvm.org.apache.flink.python.util.PythonConfigUtil + JPythonConfigUtil.configPythonOperator(j_data_stream.getExecutionEnvironment()) + + if len(fields_or_schema) == 0: + self._j_tenv.createTemporaryView(view_path, j_data_stream) + elif len(fields_or_schema) == 1 and isinstance(fields_or_schema[0], str): + self._j_tenv.createTemporaryView( + view_path, + j_data_stream, + fields_or_schema[0]) + elif len(fields_or_schema) == 1 and isinstance(fields_or_schema[0], Schema): + self._j_tenv.createTemporaryView( + view_path, + j_data_stream, + fields_or_schema[0]._j_schema) + elif (len(fields_or_schema) > 0 and + all(isinstance(elem, Expression) for elem in fields_or_schema)): + self._j_tenv.createTemporaryView( + view_path, + j_data_stream, + to_expression_jarray(fields_or_schema)) + else: + raise ValueError("Invalid arguments for 'fields': %r" % + ','.join([repr(item) for item in fields_or_schema])) def add_python_file(self, file_path: str): """ @@ -1189,14 +1194,12 @@ def add_python_file(self, file_path: str): .. versionadded:: 1.10.0 """ jvm = get_gateway().jvm - python_files = self.get_config().get_configuration().get_string( - jvm.PythonOptions.PYTHON_FILES.key(), None) + python_files = self.get_config().get(jvm.PythonOptions.PYTHON_FILES.key(), None) if python_files is not None: python_files = jvm.PythonDependencyUtils.FILE_DELIMITER.join([file_path, python_files]) else: python_files = file_path - self.get_config().get_configuration().set_string( - jvm.PythonOptions.PYTHON_FILES.key(), python_files) + self.get_config().set(jvm.PythonOptions.PYTHON_FILES.key(), python_files) def set_python_requirements(self, requirements_file_path: str, @@ -1239,7 +1242,7 @@ def set_python_requirements(self, if requirements_cache_dir is not None: python_requirements = jvm.PythonDependencyUtils.PARAM_DELIMITER.join( [python_requirements, requirements_cache_dir]) - self.get_config().get_configuration().set_string( + self.get_config().set( jvm.PythonOptions.PYTHON_REQUIREMENTS.key(), python_requirements) def add_python_archive(self, archive_path: str, target_dir: str = None): @@ -1297,15 +1300,13 @@ def add_python_archive(self, archive_path: str, target_dir: str = None): if target_dir is not None: archive_path = jvm.PythonDependencyUtils.PARAM_DELIMITER.join( [archive_path, target_dir]) - python_archives = self.get_config().get_configuration().get_string( - jvm.PythonOptions.PYTHON_ARCHIVES.key(), None) + python_archives = self.get_config().get(jvm.PythonOptions.PYTHON_ARCHIVES.key(), None) if python_archives is not None: python_files = jvm.PythonDependencyUtils.FILE_DELIMITER.join( [python_archives, archive_path]) else: python_files = archive_path - self.get_config().get_configuration().set_string( - jvm.PythonOptions.PYTHON_ARCHIVES.key(), python_files) + self.get_config().set(jvm.PythonOptions.PYTHON_ARCHIVES.key(), python_files) def from_elements(self, elements: Iterable, schema: Union[DataType, List[str]] = None, verify_schema: bool = True) -> Table: @@ -1539,10 +1540,14 @@ def _set_python_executable_for_local_executor(self): def _add_jars_to_j_env_config(self, config_key): jvm = get_gateway().jvm - jar_urls = self.get_config().get_configuration().get_string(config_key, None) + jar_urls = self.get_config().get(config_key, None) if jar_urls is not None: # normalize - jar_urls_list = [jvm.java.net.URL(url).toString() for url in jar_urls.split(";")] + jar_urls_list = [] + for url in jar_urls.split(";"): + url = url.strip() + if url != "": + jar_urls_list.append(jvm.java.net.URL(url).toString()) j_configuration = get_j_env_configuration(self._get_j_env()) if j_configuration.containsKey(config_key): for url in j_configuration.getString(config_key, "").split(";"): @@ -1620,14 +1625,10 @@ def _config_chaining_optimization(self): def _open(self): # start BeamFnLoopbackWorkerPoolServicer when executed in MiniCluster def startup_loopback_server(): - from pyflink.common import Configuration from pyflink.fn_execution.beam.beam_worker_pool_service import \ BeamFnLoopbackWorkerPoolServicer - - j_configuration = get_j_env_configuration(self._get_j_env()) - config = Configuration(j_configuration=j_configuration) - config.set_string( - "PYFLINK_LOOPBACK_SERVER_ADDRESS", BeamFnLoopbackWorkerPoolServicer().start()) + self.get_config().set("python.loopback-server.address", + BeamFnLoopbackWorkerPoolServicer().start()) python_worker_execution_mode = os.environ.get('_python_worker_execution_mode') @@ -1653,7 +1654,6 @@ def __init__(self, j_tenv): @staticmethod def create(stream_execution_environment: StreamExecutionEnvironment = None, # type: ignore - table_config: TableConfig = None, environment_settings: EnvironmentSettings = None) -> 'StreamTableEnvironment': """ Creates a :class:`~pyflink.table.StreamTableEnvironment`. @@ -1664,12 +1664,14 @@ def create(stream_execution_environment: StreamExecutionEnvironment = None, # t # create with StreamExecutionEnvironment. >>> env = StreamExecutionEnvironment.get_execution_environment() >>> table_env = StreamTableEnvironment.create(env) - # create with StreamExecutionEnvironment and TableConfig. - >>> table_config = TableConfig() - >>> table_config.set_null_check(False) - >>> table_env = StreamTableEnvironment.create(env, table_config) # create with StreamExecutionEnvironment and EnvironmentSettings. - >>> environment_settings = EnvironmentSettings.in_streaming_mode() + >>> configuration = Configuration() + >>> configuration.set_string('execution.buffer-timeout', '1 min') + >>> environment_settings = EnvironmentSettings \\ + ... .new_instance() \\ + ... .in_streaming_mode() \\ + ... .with_configuration(configuration) \\ + ... .build() >>> table_env = StreamTableEnvironment.create( ... env, environment_settings=environment_settings) # create with EnvironmentSettings. @@ -1679,27 +1681,15 @@ def create(stream_execution_environment: StreamExecutionEnvironment = None, # t :param stream_execution_environment: The :class:`~pyflink.datastream.StreamExecutionEnvironment` of the TableEnvironment. - :param table_config: The configuration of the TableEnvironment, optional. :param environment_settings: The environment settings used to instantiate the TableEnvironment. :return: The StreamTableEnvironment created from given StreamExecutionEnvironment and configuration. """ if stream_execution_environment is None and \ - table_config is None and \ environment_settings is None: raise ValueError("No argument found, the param 'stream_execution_environment' " "or 'environment_settings' is required.") - elif stream_execution_environment is None and \ - table_config is not None and \ - environment_settings is None: - raise ValueError("Only the param 'table_config' is found, " - "the param 'stream_execution_environment' is also required.") - if table_config is not None and \ - environment_settings is not None: - raise ValueError("The param 'table_config' and " - "'environment_settings' cannot be used at the same time") - gateway = get_gateway() if environment_settings is not None: if stream_execution_environment is None: @@ -1710,13 +1700,9 @@ def create(stream_execution_environment: StreamExecutionEnvironment = None, # t stream_execution_environment._j_stream_execution_environment, environment_settings._j_environment_settings) else: - if table_config is not None: - j_tenv = gateway.jvm.StreamTableEnvironment.create( - stream_execution_environment._j_stream_execution_environment, - table_config._j_table_config) - else: - j_tenv = gateway.jvm.StreamTableEnvironment.create( - stream_execution_environment._j_stream_execution_environment) + j_tenv = gateway.jvm.StreamTableEnvironment.create( + stream_execution_environment._j_stream_execution_environment) + return StreamTableEnvironment(j_tenv) def from_data_stream(self, diff --git a/flink-python/pyflink/table/tests/test_dependency.py b/flink-python/pyflink/table/tests/test_dependency.py index b180c729b7de9..121fe87338063 100644 --- a/flink-python/pyflink/table/tests/test_dependency.py +++ b/flink-python/pyflink/table/tests/test_dependency.py @@ -93,19 +93,10 @@ def add_from_file(i): @pytest.mark.skipif(sys.version_info < (3, 7), reason="requires python3.7") -class EmbeddedMultiThreadDependencyTests(DependencyTests, PyFlinkStreamTableTestCase): +class EmbeddedThreadDependencyTests(DependencyTests, PyFlinkStreamTableTestCase): def setUp(self): - super(EmbeddedMultiThreadDependencyTests, self).setUp() - self.t_env.get_config().get_configuration().set_string("python.execution-mode", - "multi-thread") - - -@pytest.mark.skipif(sys.version_info < (3, 8), reason="requires python3.8") -class EmbeddedSubInterpreterDependencyTests(DependencyTests, PyFlinkStreamTableTestCase): - def setUp(self): - super(EmbeddedSubInterpreterDependencyTests, self).setUp() - self.t_env.get_config().get_configuration().set_string("python.execution-mode", - "sub-interpreter") + super(EmbeddedThreadDependencyTests, self).setUp() + self.t_env.get_config().set("python.execution-mode", "thread") class BatchDependencyTests(DependencyTests, PyFlinkBatchTableTestCase): diff --git a/flink-python/pyflink/table/tests/test_environment_settings.py b/flink-python/pyflink/table/tests/test_environment_settings.py index 8722904a68429..1a2f1ed612e92 100644 --- a/flink-python/pyflink/table/tests/test_environment_settings.py +++ b/flink-python/pyflink/table/tests/test_environment_settings.py @@ -50,7 +50,7 @@ def test_with_built_in_catalog_name(self): gateway = get_gateway() - DEFAULT_BUILTIN_CATALOG = gateway.jvm.EnvironmentSettings.DEFAULT_BUILTIN_CATALOG + DEFAULT_BUILTIN_CATALOG = gateway.jvm.TableConfigOptions.TABLE_CATALOG_NAME.defaultValue() builder = EnvironmentSettings.new_instance() @@ -67,7 +67,7 @@ def test_with_built_in_database_name(self): gateway = get_gateway() - DEFAULT_BUILTIN_DATABASE = gateway.jvm.EnvironmentSettings.DEFAULT_BUILTIN_DATABASE + DEFAULT_BUILTIN_DATABASE = gateway.jvm.TableConfigOptions.TABLE_DATABASE_NAME.defaultValue() builder = EnvironmentSettings.new_instance() diff --git a/flink-python/pyflink/table/tests/test_pandas_conversion.py b/flink-python/pyflink/table/tests/test_pandas_conversion.py index e4b6d60cc2658..89fc485dd5304 100644 --- a/flink-python/pyflink/table/tests/test_pandas_conversion.py +++ b/flink-python/pyflink/table/tests/test_pandas_conversion.py @@ -181,7 +181,7 @@ class BatchPandasConversionTests(PandasConversionTests, class StreamPandasConversionTests(PandasConversionITTests, PyFlinkStreamTableTestCase): def test_to_pandas_with_event_time(self): - self.t_env.get_config().get_configuration().set_string("parallelism.default", "1") + self.t_env.get_config().set("parallelism.default", "1") # create source file path import tempfile import os @@ -199,7 +199,7 @@ def test_to_pandas_with_event_time(self): for ele in data: fd.write(ele + '\n') - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "pipeline.time-characteristic", "EventTime") source_table = """ diff --git a/flink-python/pyflink/table/tests/test_pandas_udaf.py b/flink-python/pyflink/table/tests/test_pandas_udaf.py index 2a9595129c5fb..d658d087525c6 100644 --- a/flink-python/pyflink/table/tests/test_pandas_udaf.py +++ b/flink-python/pyflink/table/tests/test_pandas_udaf.py @@ -102,7 +102,7 @@ def test_group_aggregate_with_aux_group(self): ['a', 'b', 'c', 'd'], [DataTypes.TINYINT(), DataTypes.INT(), DataTypes.FLOAT(), DataTypes.INT()]) self.t_env.register_table_sink("Results", table_sink) - self.t_env.get_config().get_configuration().set_string('python.metric.enabled', 'true') + self.t_env.get_config().set('python.metric.enabled', 'true') self.t_env.register_function("max_add", udaf(MaxAdd(), result_type=DataTypes.INT(), func_type="pandas")) @@ -300,7 +300,7 @@ def test_sliding_group_window_over_time(self): fd.write(ele + '\n') from pyflink.table.window import Slide - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "pipeline.time-characteristic", "EventTime") self.t_env.register_function("mean_udaf", mean_udaf) @@ -349,7 +349,7 @@ def test_sliding_group_window_over_time(self): os.remove(source_path) def test_sliding_group_window_over_proctime(self): - self.t_env.get_config().get_configuration().set_string("parallelism.default", "1") + self.t_env.get_config().set("parallelism.default", "1") from pyflink.table.window import Slide self.t_env.register_function("mean_udaf", mean_udaf) @@ -379,7 +379,7 @@ def test_sliding_group_window_over_proctime(self): self.assertTrue(result[0][1].year > 1970) def test_sliding_group_window_over_count(self): - self.t_env.get_config().get_configuration().set_string("parallelism.default", "1") + self.t_env.get_config().set("parallelism.default", "1") # create source file path import tempfile import os @@ -399,7 +399,7 @@ def test_sliding_group_window_over_count(self): fd.write(ele + '\n') from pyflink.table.window import Slide - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "pipeline.time-characteristic", "ProcessingTime") self.t_env.register_function("mean_udaf", mean_udaf) @@ -454,7 +454,7 @@ def test_tumbling_group_window_over_time(self): fd.write(ele + '\n') from pyflink.table.window import Tumble - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "pipeline.time-characteristic", "EventTime") self.t_env.register_function("mean_udaf", mean_udaf) @@ -500,7 +500,7 @@ def test_tumbling_group_window_over_time(self): os.remove(source_path) def test_tumbling_group_window_over_count(self): - self.t_env.get_config().get_configuration().set_string("parallelism.default", "1") + self.t_env.get_config().set("parallelism.default", "1") # create source file path import tempfile import os @@ -521,7 +521,7 @@ def test_tumbling_group_window_over_count(self): fd.write(ele + '\n') from pyflink.table.window import Tumble - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "pipeline.time-characteristic", "ProcessingTime") self.t_env.register_function("mean_udaf", mean_udaf) @@ -577,7 +577,7 @@ def test_row_time_over_range_window_aggregate_function(self): max_add_min_udaf = udaf(lambda a: a.max() + a.min(), result_type=DataTypes.SMALLINT(), func_type='pandas') - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "pipeline.time-characteristic", "EventTime") self.t_env.register_function("mean_udaf", mean_udaf) self.t_env.register_function("max_add_min_udaf", max_add_min_udaf) @@ -645,7 +645,7 @@ def test_row_time_over_rows_window_aggregate_function(self): max_add_min_udaf = udaf(lambda a: a.max() + a.min(), result_type=DataTypes.SMALLINT(), func_type='pandas') - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "pipeline.time-characteristic", "EventTime") self.t_env.register_function("mean_udaf", mean_udaf) self.t_env.register_function("max_add_min_udaf", max_add_min_udaf) @@ -713,8 +713,8 @@ def test_proc_time_over_rows_window_aggregate_function(self): max_add_min_udaf = udaf(lambda a: a.max() + a.min(), result_type=DataTypes.SMALLINT(), func_type='pandas') - self.t_env.get_config().get_configuration().set_string("parallelism.default", "1") - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set("parallelism.default", "1") + self.t_env.get_config().set( "pipeline.time-characteristic", "ProcessingTime") self.t_env.register_function("mean_udaf", mean_udaf) self.t_env.register_function("max_add_min_udaf", max_add_min_udaf) @@ -807,7 +807,7 @@ def test_execute_over_aggregate_from_json_plan(self): max_add_min_udaf = udaf(lambda a: a.max() + a.min(), result_type=DataTypes.SMALLINT(), func_type='pandas') - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "pipeline.time-characteristic", "EventTime") self.t_env.create_temporary_system_function("mean_udaf", mean_udaf) self.t_env.create_temporary_system_function("max_add_min_udaf", max_add_min_udaf) 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 03b99b41b12a2..8ddfe3af45959 100644 --- a/flink-python/pyflink/table/tests/test_row_based_operation.py +++ b/flink-python/pyflink/table/tests/test_row_based_operation.py @@ -280,10 +280,10 @@ def test_flat_aggregate(self): def test_flat_aggregate_list_view(self): import pandas as pd my_concat = udtaf(ListViewConcatTableAggregateFunction()) - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "python.fn-execution.bundle.size", "2") # trigger the cache eviction in a bundle. - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "python.state.cache-size", "2") t = self.t_env.from_elements([(1, 'Hi', 'Hello'), (3, 'Hi', 'hi'), diff --git a/flink-python/pyflink/table/tests/test_table_config.py b/flink-python/pyflink/table/tests/test_table_config.py index bb489ccd67c90..4bada304ca3f8 100644 --- a/flink-python/pyflink/table/tests/test_table_config.py +++ b/flink-python/pyflink/table/tests/test_table_config.py @@ -89,9 +89,9 @@ def test_get_set_null_check(self): def test_get_configuration(self): table_config = TableConfig.get_default() - table_config.get_configuration().set_string("k1", "v1") + table_config.set("k1", "v1") - self.assertEqual(table_config.get_configuration().get_string("k1", ""), "v1") + self.assertEqual(table_config.get("k1", ""), "v1") def test_add_configuration(self): table_config = TableConfig.get_default() @@ -100,7 +100,7 @@ def test_add_configuration(self): table_config.add_configuration(configuration) - self.assertEqual(table_config.get_configuration().get_string("k1", ""), "v1") + self.assertEqual(table_config.get("k1", ""), "v1") def test_get_set_sql_dialect(self): table_config = TableConfig.get_default() diff --git a/flink-python/pyflink/table/tests/test_table_config_completeness.py b/flink-python/pyflink/table/tests/test_table_config_completeness.py index 396f5d7e5c7d5..d8b9bfae2a59a 100644 --- a/flink-python/pyflink/table/tests/test_table_config_completeness.py +++ b/flink-python/pyflink/table/tests/test_table_config_completeness.py @@ -37,7 +37,8 @@ def java_class(cls): @classmethod def excluded_methods(cls): # internal interfaces, no need to expose to users. - return {'getPlannerConfig', 'setPlannerConfig', 'addJobParameter'} + return {'getPlannerConfig', 'setPlannerConfig', 'addJobParameter', + 'setRootConfiguration', 'getRootConfiguration', 'getOptional'} @classmethod def java_method_name(cls, python_method_name): 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 ddf31c0e27b72..97eb99bccc407 100644 --- a/flink-python/pyflink/table/tests/test_table_environment_api.py +++ b/flink-python/pyflink/table/tests/test_table_environment_api.py @@ -243,7 +243,7 @@ def setUp(self) -> None: self.env.set_parallelism(2) config = get_j_env_configuration(self.env._j_stream_execution_environment) config.setString("akka.ask.timeout", "20 s") - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "python.fn-execution.bundle.size", "1") self.test_sink = DataStreamTestSinkFunction() diff --git a/flink-python/pyflink/table/tests/test_udaf.py b/flink-python/pyflink/table/tests/test_udaf.py index 2fe4948bcd33e..68125c979f906 100644 --- a/flink-python/pyflink/table/tests/test_udaf.py +++ b/flink-python/pyflink/table/tests/test_udaf.py @@ -247,10 +247,10 @@ def test_double_aggregate(self): self.t_env.create_temporary_function("my_sum", SumAggregateFunction()) # trigger the finish bundle more frequently to ensure testing the communication # between RemoteKeyedStateBackend and the StateGrpcService. - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "python.fn-execution.bundle.size", "2") # trigger the cache eviction in a bundle. - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "python.state.cache-size", "1") t = self.t_env.from_elements([(1, 'Hi', 'Hello'), (3, 'Hi', 'hi'), @@ -263,7 +263,7 @@ def test_double_aggregate(self): pd.DataFrame([[3, 12, 12, 12.0]], columns=['a', 'b', 'c', 'd'])) def test_mixed_with_built_in_functions_with_retract(self): - self.t_env.get_config().get_configuration().set_string("parallelism.default", "1") + self.t_env.get_config().set("parallelism.default", "1") self.t_env.create_temporary_system_function( "concat", ConcatAggregateFunction()) @@ -311,7 +311,7 @@ def test_mixed_with_built_in_functions_with_retract(self): self.assertEqual(result[len(result) - 1], expected) def test_mixed_with_built_in_functions_without_retract(self): - self.t_env.get_config().get_configuration().set_string("parallelism.default", "1") + self.t_env.get_config().set("parallelism.default", "1") self.t_env.create_temporary_system_function( "concat", ConcatAggregateFunction()) @@ -357,10 +357,10 @@ def test_using_decorator(self): def test_list_view(self): my_concat = udaf(ListViewConcatAggregateFunction()) - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "python.fn-execution.bundle.size", "2") # trigger the cache eviction in a bundle. - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "python.state.cache-size", "2") t = self.t_env.from_elements([(1, 'Hi', 'Hello'), (3, 'Hi', 'hi'), @@ -380,14 +380,14 @@ def test_list_view(self): def test_map_view(self): my_count = udaf(CountDistinctAggregateFunction()) self.t_env.get_config().set_idle_state_retention(datetime.timedelta(days=1)) - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "python.fn-execution.bundle.size", "2") # trigger the cache eviction in a bundle. - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "python.state.cache-size", "1") - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "python.map-state.read-cache-size", "1") - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "python.map-state.write-cache-size", "1") t = self.t_env.from_elements( [(1, 'Hi_', 'hi'), @@ -414,10 +414,10 @@ def test_map_view(self): def test_data_view_clear(self): my_count = udaf(CountDistinctAggregateFunction()) self.t_env.get_config().set_idle_state_retention(datetime.timedelta(days=1)) - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "python.fn-execution.bundle.size", "2") # trigger the cache eviction in a bundle. - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "python.state.cache-size", "1") t = self.t_env.from_elements( [(2, 'hello', 'hello'), @@ -431,16 +431,16 @@ def test_data_view_clear(self): def test_map_view_iterate(self): test_iterate = udaf(TestIterateAggregateFunction()) self.t_env.get_config().set_idle_state_retention(datetime.timedelta(days=1)) - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "python.fn-execution.bundle.size", "2") # trigger the cache eviction in a bundle. - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "python.state.cache-size", "2") - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "python.map-state.read-cache-size", "2") - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "python.map-state.write-cache-size", "2") - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "python.map-state.iterate-response-batch-size", "2") t = self.t_env.from_elements( [(1, 'Hi_', 'hi'), @@ -508,12 +508,12 @@ def test_distinct_and_filter(self): def test_clean_state(self): self.t_env.register_function("my_count", CountAggregateFunction()) - self.t_env.get_config().get_configuration().set_string("parallelism.default", "1") - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set("parallelism.default", "1") + self.t_env.get_config().set( "python.fn-execution.bundle.size", "1") - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "python.state.cache-size", "0") - self.t_env.get_config().get_configuration().set_string( + self.t_env.get_config().set( "table.exec.state.ttl", "2ms") self.t_env.execute_sql(""" @@ -587,7 +587,7 @@ def test_tumbling_group_window_over_time(self): "+I[1, 2018-03-11 04:00:00.0, 2018-03-11 05:00:00.0, 1, 1]"]) def test_tumbling_group_window_over_count(self): - self.t_env.get_config().get_configuration().set_string("parallelism.default", "1") + self.t_env.get_config().set("parallelism.default", "1") # create source file path tmp_dir = self.tempdir data = [ @@ -701,7 +701,7 @@ def test_sliding_group_window_over_time(self): "+I[1, 2018-03-11 04:00:00.0, 2018-03-11 05:00:00.0, 8]"]) def test_sliding_group_window_over_count(self): - self.t_env.get_config().get_configuration().set_string("parallelism.default", "1") + self.t_env.get_config().set("parallelism.default", "1") # create source file path tmp_dir = self.tempdir data = [ diff --git a/flink-python/pyflink/table/tests/test_udf.py b/flink-python/pyflink/table/tests/test_udf.py index 6abdee2fbc0a1..135f1d3ddc77d 100644 --- a/flink-python/pyflink/table/tests/test_udf.py +++ b/flink-python/pyflink/table/tests/test_udf.py @@ -34,7 +34,7 @@ class UserDefinedFunctionTests(object): def test_scalar_function(self): # test metric disabled. - self.t_env.get_config().get_configuration().set_string('python.metric.enabled', 'false') + self.t_env.get_config().set('python.metric.enabled', 'false') # test lambda function add_one = udf(lambda i: i + 1, result_type=DataTypes.BIGINT()) @@ -65,8 +65,7 @@ def check_memory_limit(exec_mode): DataTypes.BIGINT(), DataTypes.BIGINT(), DataTypes.BIGINT()]) self.t_env.register_table_sink("Results", table_sink) - execution_mode = self.t_env.get_config().get_configuration().get_string( - "python.execution-mode", "process") + execution_mode = self.t_env.get_config().get("python.execution-mode", "process") t = self.t_env.from_elements([(1, 2, 3), (2, 5, 6), (3, 1, 9)], ['a', 'b', 'c']) t.where(add_one(t.b) <= 3).select( @@ -221,9 +220,8 @@ def test_overwrite_builtin_function(self): self.assert_equals(actual, ["+I[2]", "+I[6]", "+I[3]"]) def test_open(self): - self.t_env.get_config().get_configuration().set_string('python.metric.enabled', 'true') - execution_mode = self.t_env.get_config().get_configuration().get_string( - "python.execution-mode", None) + self.t_env.get_config().set('python.metric.enabled', 'true') + execution_mode = self.t_env.get_config().get("python.execution-mode", None) if execution_mode == "process": subtract = udf(SubtractWithMetrics(), result_type=DataTypes.BIGINT()) @@ -797,20 +795,10 @@ class PyFlinkBatchUserDefinedFunctionTests(UserDefinedFunctionTests, @pytest.mark.skipif(sys.version_info < (3, 7), reason="requires python3.7") -class PyFlinkEmbeddedMultiThreadTests(UserDefinedFunctionTests, PyFlinkBatchTableTestCase): +class PyFlinkEmbeddedThreadTests(UserDefinedFunctionTests, PyFlinkBatchTableTestCase): def setUp(self): - super(PyFlinkEmbeddedMultiThreadTests, self).setUp() - self.t_env.get_config().get_configuration().set_string("python.execution-mode", - "multi-thread") - - -@pytest.mark.skipif(sys.version_info < (3, 8), reason="requires python3.8") -class PyFlinkEmbeddedSubInterpreterTests(UserDefinedFunctionTests, PyFlinkBatchTableTestCase): - def setUp(self): - super(PyFlinkEmbeddedSubInterpreterTests, self).setUp() - self.t_env.get_config().get_configuration().set_string("python.execution-mode", - "sub-interpreter") - + super(PyFlinkEmbeddedThreadTests, self).setUp() + self.t_env.get_config().set("python.execution-mode", "thread") # test specify the input_types @udf(input_types=[DataTypes.BIGINT(), DataTypes.BIGINT()], result_type=DataTypes.BIGINT()) diff --git a/flink-python/pyflink/testing/test_case_utils.py b/flink-python/pyflink/testing/test_case_utils.py index 3849d0c6e1ea2..c8749648160b6 100644 --- a/flink-python/pyflink/testing/test_case_utils.py +++ b/flink-python/pyflink/testing/test_case_utils.py @@ -135,8 +135,8 @@ class PyFlinkStreamTableTestCase(PyFlinkTestCase): def setUp(self): 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( + self.t_env.get_config().set("parallelism.default", "2") + self.t_env.get_config().set( "python.fn-execution.bundle.size", "1") @@ -148,8 +148,8 @@ class PyFlinkBatchTableTestCase(PyFlinkTestCase): def setUp(self): 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( + self.t_env.get_config().set("parallelism.default", "2") + self.t_env.get_config().set( "python.fn-execution.bundle.size", "1") diff --git a/flink-python/pyflink/version.py b/flink-python/pyflink/version.py index c29e6a8a03be9..054b900b4ca4e 100644 --- a/flink-python/pyflink/version.py +++ b/flink-python/pyflink/version.py @@ -20,4 +20,4 @@ The pyflink version will be consistent with the flink version and follow the PEP440. .. seealso:: https://www.python.org/dev/peps/pep-0440 """ -__version__ = "1.15.dev0" +__version__ = "1.15.0" diff --git a/flink-python/setup.py b/flink-python/setup.py index 6bd3f97ed986d..36facf77f4bbf 100644 --- a/flink-python/setup.py +++ b/flink-python/setup.py @@ -313,7 +313,7 @@ def extracted_output_files(base_dir, file_path, output_directory): 'pandas>=1.0,<1.2.0', 'pyarrow>=0.15.1,<3.0.0', 'pytz>=2018.3', 'numpy>=1.14.3,<1.20', 'fastavro>=0.21.4,<0.24', 'requests>=2.26.0', 'protobuf<3.18', - 'pemja==0.1.2;python_full_version >= "3.7"', + 'pemja==0.1.4;python_full_version >= "3.7"', apache_flink_libraries_dependency], cmdclass={'build_ext': build_ext}, tests_require=['pytest==4.4.1'], diff --git a/flink-python/src/main/java/org/apache/flink/client/python/PythonDriver.java b/flink-python/src/main/java/org/apache/flink/client/python/PythonDriver.java index 57b1eb2f32d51..a25ee91d27069 100644 --- a/flink-python/src/main/java/org/apache/flink/client/python/PythonDriver.java +++ b/flink-python/src/main/java/org/apache/flink/client/python/PythonDriver.java @@ -154,10 +154,14 @@ public static void main(String[] args) throws Throwable { */ static List constructPythonCommands(final PythonDriverOptions pythonDriverOptions) { final List commands = new ArrayList<>(); + commands.add("-m"); if (pythonDriverOptions.getEntryPointScript().isPresent()) { - commands.add(pythonDriverOptions.getEntryPointScript().get()); + String pythonFileName = pythonDriverOptions.getEntryPointScript().get(); + commands.add( + pythonFileName.substring( + pythonFileName.lastIndexOf(File.separator) + 1, + pythonFileName.lastIndexOf(".py"))); } else { - commands.add("-m"); commands.add(pythonDriverOptions.getEntryPointModule()); } commands.addAll(pythonDriverOptions.getProgramArgs()); diff --git a/flink-python/src/main/java/org/apache/flink/client/python/PythonDriverOptions.java b/flink-python/src/main/java/org/apache/flink/client/python/PythonDriverOptions.java index bec0889d05b83..792af77b4212d 100644 --- a/flink-python/src/main/java/org/apache/flink/client/python/PythonDriverOptions.java +++ b/flink-python/src/main/java/org/apache/flink/client/python/PythonDriverOptions.java @@ -29,11 +29,11 @@ /** Options for the {@link PythonDriver}. */ final class PythonDriverOptions { - @Nullable private String entryPointModule; + @Nullable private final String entryPointModule; - @Nullable private String entryPointScript; + @Nullable private final String entryPointScript; - @Nonnull private List programArgs; + @Nonnull private final List programArgs; @Nullable String getEntryPointModule() { diff --git a/flink-python/src/main/java/org/apache/flink/client/python/PythonDriverOptionsParserFactory.java b/flink-python/src/main/java/org/apache/flink/client/python/PythonDriverOptionsParserFactory.java index 6e5663e129eb6..ded5577642aa3 100644 --- a/flink-python/src/main/java/org/apache/flink/client/python/PythonDriverOptionsParserFactory.java +++ b/flink-python/src/main/java/org/apache/flink/client/python/PythonDriverOptionsParserFactory.java @@ -54,6 +54,12 @@ public PythonDriverOptions createResult(@Nonnull CommandLine commandLine) throw new FlinkParseException("Cannot use options -py and -pym simultaneously."); } else if (commandLine.hasOption(PY_OPTION.getOpt())) { entryPointScript = commandLine.getOptionValue(PY_OPTION.getOpt()); + if (!entryPointScript.endsWith(".py")) { + throw new FlinkParseException( + String.format( + "It only accepts Python file which ends with '.py' for option '-py', got '%s'.", + entryPointScript)); + } } else if (commandLine.hasOption(PYMODULE_OPTION.getOpt())) { entryPointModule = commandLine.getOptionValue(PYMODULE_OPTION.getOpt()); } else { diff --git a/flink-python/src/main/java/org/apache/flink/client/python/PythonEnvUtils.java b/flink-python/src/main/java/org/apache/flink/client/python/PythonEnvUtils.java index ca7d9907c8b06..261aa82dc7656 100644 --- a/flink-python/src/main/java/org/apache/flink/client/python/PythonEnvUtils.java +++ b/flink-python/src/main/java/org/apache/flink/client/python/PythonEnvUtils.java @@ -484,9 +484,9 @@ public static void shutdownPythonProcess(Process pythonProcess, long timeoutMill /** The shutdown hook used to destroy the Python process. */ public static class PythonProcessShutdownHook extends Thread { - private Process process; - private GatewayServer gatewayServer; - private String tmpDir; + private final Process process; + private final GatewayServer gatewayServer; + private final String tmpDir; public PythonProcessShutdownHook( Process process, GatewayServer gatewayServer, String tmpDir) { diff --git a/flink-python/src/main/java/org/apache/flink/client/python/PythonFunctionFactory.java b/flink-python/src/main/java/org/apache/flink/client/python/PythonFunctionFactory.java index 158baba841f6d..e7c4d8a3755e7 100644 --- a/flink-python/src/main/java/org/apache/flink/client/python/PythonFunctionFactory.java +++ b/flink-python/src/main/java/org/apache/flink/client/python/PythonFunctionFactory.java @@ -22,6 +22,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.python.util.PythonDependencyUtils; +import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.functions.python.PythonFunction; import org.apache.flink.util.FileUtils; import org.apache.flink.util.concurrent.ExecutorThreadFactory; @@ -124,7 +125,11 @@ static PythonFunction getPythonFunction( Configuration mergedConfig = new Configuration( ExecutionEnvironment.getExecutionEnvironment().getConfiguration()); - PythonDependencyUtils.merge(mergedConfig, (Configuration) config); + if (config instanceof TableConfig) { + PythonDependencyUtils.merge(mergedConfig, ((TableConfig) config).getConfiguration()); + } else { + PythonDependencyUtils.merge(mergedConfig, (Configuration) config); + } PythonFunctionFactory pythonFunctionFactory = PYTHON_FUNCTION_FACTORY_CACHE.get(CacheKey.of(mergedConfig, classLoader)); ensureCacheCleanupExecutorServiceStarted(); diff --git a/flink-python/src/main/java/org/apache/flink/python/PythonConfig.java b/flink-python/src/main/java/org/apache/flink/python/PythonConfig.java index 5195f1f78fa13..e5c24f9beba8b 100644 --- a/flink-python/src/main/java/org/apache/flink/python/PythonConfig.java +++ b/flink-python/src/main/java/org/apache/flink/python/PythonConfig.java @@ -19,152 +19,101 @@ package org.apache.flink.python; import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigUtils; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.python.util.PythonDependencyUtils; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.util.Preconditions; -import javax.annotation.Nullable; - -import java.io.Serializable; +import java.time.ZoneId; +import java.util.ArrayList; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Optional; +import static org.apache.flink.python.PythonOptions.PYTHON_LOOPBACK_SERVER_ADDRESS; + /** Configurations for the Python job which are used at run time. */ @Internal -public class PythonConfig implements Serializable { - - private static final long serialVersionUID = 1L; - - /** Max number of elements to include in a bundle. */ - private final int maxBundleSize; - - /** Max duration of a bundle. */ - private final long maxBundleTimeMills; +public class PythonConfig implements ReadableConfig { - /** Max number of elements to include in an arrow batch. */ - private final int maxArrowBatchSize; + private static final List> PYTHON_CONFIG_OPTIONS; - /** - * The python files uploaded by pyflink.table.TableEnvironment#add_python_file() or command line - * option "-pyfs". The key is the file key in distribute cache and the value is the - * corresponding origin file name. - */ - private final Map pythonFilesInfo; - - /** - * The file key of the requirements file in distribute cache. It is specified by - * pyflink.table.TableEnvironment#set_python_requirements() or command line option "-pyreq". - */ - @Nullable private final String pythonRequirementsFileInfo; - - /** - * The file key of the requirements cached directory in distribute cache. It is specified by - * pyflink.table.TableEnvironment#set_python_requirements() or command line option "-pyreq". It - * is used to support installing python packages offline. - */ - @Nullable private final String pythonRequirementsCacheDirInfo; + static { + PYTHON_CONFIG_OPTIONS = + new ArrayList<>(ConfigUtils.getAllConfigOptions(PythonOptions.class)); + } /** - * The python archives uploaded by pyflink.table.TableEnvironment#add_python_archive() or - * command line option "-pyarch". The key is the file key of the archives in distribute cache - * and the value is the name of the directory to extract to. + * Configuration adopted from the outer layer, e.g. flink-conf.yaml, command line arguments, + * TableConfig, etc. */ - private final Map pythonArchivesInfo; + private final ReadableConfig configuration; /** - * The path of the python interpreter (e.g. /usr/local/bin/python) specified by - * pyflink.table.TableConfig#set_python_executable() or command line option "-pyexec". + * Configuration generated in the dependency management mechanisms. See {@link + * PythonDependencyUtils.PythonDependencyManager} for more details. */ - private final String pythonExec; - - /** Whether metric is enabled. */ - private final boolean metricEnabled; - - /** Whether to use managed memory for the Python worker. */ - private final boolean isUsingManagedMemory; - - /** The Configuration that contains execution configs and dependencies info. */ - private final Configuration config; - - /** Whether profile is enabled. */ - private final boolean profileEnabled; - - /** Execution Mode. */ - private final String executionMode; - - public PythonConfig(Configuration config) { - this.config = config; - maxBundleSize = config.get(PythonOptions.MAX_BUNDLE_SIZE); - maxBundleTimeMills = config.get(PythonOptions.MAX_BUNDLE_TIME_MILLS); - maxArrowBatchSize = config.get(PythonOptions.MAX_ARROW_BATCH_SIZE); - pythonFilesInfo = - config.getOptional(PythonDependencyUtils.PYTHON_FILES).orElse(new HashMap<>()); - pythonRequirementsFileInfo = - config.getOptional(PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE) - .orElse(new HashMap<>()) - .get(PythonDependencyUtils.FILE); - pythonRequirementsCacheDirInfo = - config.getOptional(PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE) - .orElse(new HashMap<>()) - .get(PythonDependencyUtils.CACHE); - pythonArchivesInfo = - config.getOptional(PythonDependencyUtils.PYTHON_ARCHIVES).orElse(new HashMap<>()); - pythonExec = config.get(PythonOptions.PYTHON_EXECUTABLE); - metricEnabled = config.getBoolean(PythonOptions.PYTHON_METRIC_ENABLED); - isUsingManagedMemory = config.getBoolean(PythonOptions.USE_MANAGED_MEMORY); - profileEnabled = config.getBoolean(PythonOptions.PYTHON_PROFILE_ENABLED); - executionMode = config.getString(PythonOptions.PYTHON_EXECUTION_MODE); - } - - public int getMaxBundleSize() { - return maxBundleSize; - } + private final ReadableConfig pythonDependencyConfiguration; - public long getMaxBundleTimeMills() { - return maxBundleTimeMills; + public PythonConfig( + ReadableConfig configuration, ReadableConfig pythonDependencyConfiguration) { + this.configuration = Preconditions.checkNotNull(configuration); + this.pythonDependencyConfiguration = + Preconditions.checkNotNull(pythonDependencyConfiguration); } - public int getMaxArrowBatchSize() { - return maxArrowBatchSize; + @Override + public T get(ConfigOption option) { + return pythonDependencyConfiguration + .getOptional(option) + .orElseGet(() -> configuration.get(option)); } - public Map getPythonFilesInfo() { - return pythonFilesInfo; + @Override + public Optional getOptional(ConfigOption option) { + final Optional value = pythonDependencyConfiguration.getOptional(option); + if (value.isPresent()) { + return value; + } + return configuration.getOptional(option); } - public Optional getPythonRequirementsFileInfo() { - return Optional.ofNullable(pythonRequirementsFileInfo); - } - - public Optional getPythonRequirementsCacheDirInfo() { - return Optional.ofNullable(pythonRequirementsCacheDirInfo); - } - - public Map getPythonArchivesInfo() { - return pythonArchivesInfo; - } - - public String getPythonExec() { - return pythonExec; - } - - public String getExecutionMode() { - return executionMode; - } + public Configuration toConfiguration() { + final Configuration config = new Configuration(); + PYTHON_CONFIG_OPTIONS.forEach( + option -> + getOptional((ConfigOption) option) + .ifPresent(v -> config.set((ConfigOption) option, v))); + + // prepare the job options + Map jobOptions = config.get(PythonOptions.PYTHON_JOB_OPTIONS); + if (jobOptions == null) { + jobOptions = new HashMap<>(); + config.set(PythonOptions.PYTHON_JOB_OPTIONS, jobOptions); + } + jobOptions.put("TABLE_LOCAL_TIME_ZONE", getLocalTimeZone(configuration).getId()); + if (config.contains(PYTHON_LOOPBACK_SERVER_ADDRESS)) { + jobOptions.put( + "PYTHON_LOOPBACK_SERVER_ADDRESS", config.get(PYTHON_LOOPBACK_SERVER_ADDRESS)); + } - public boolean isMetricEnabled() { - return metricEnabled; - } - - public boolean isProfileEnabled() { - return profileEnabled; - } - - public boolean isUsingManagedMemory() { - return isUsingManagedMemory; + return config; } - public Configuration getConfig() { - return config; + /** + * Returns the current session time zone id. It is used when converting to/from {@code TIMESTAMP + * WITH LOCAL TIME ZONE}. + * + * @see org.apache.flink.table.types.logical.LocalZonedTimestampType + */ + private static ZoneId getLocalTimeZone(ReadableConfig config) { + String zone = config.get(TableConfigOptions.LOCAL_TIME_ZONE); + return TableConfigOptions.LOCAL_TIME_ZONE.defaultValue().equals(zone) + ? ZoneId.systemDefault() + : ZoneId.of(zone); } } diff --git a/flink-python/src/main/java/org/apache/flink/python/PythonFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/python/PythonFunctionRunner.java index 710a04605555d..4a4174a768d74 100644 --- a/flink-python/src/main/java/org/apache/flink/python/PythonFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/python/PythonFunctionRunner.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ReadableConfig; /** The base interface of runner which is responsible for the execution of Python functions. */ @Internal @@ -28,7 +29,7 @@ public interface PythonFunctionRunner { /** * Prepares the Python function runner, such as preparing the Python execution environment, etc. */ - void open(PythonConfig config) throws Exception; + void open(ReadableConfig config) throws Exception; /** Tear-down the Python function runner. */ void close() throws Exception; diff --git a/flink-python/src/main/java/org/apache/flink/python/PythonOptions.java b/flink-python/src/main/java/org/apache/flink/python/PythonOptions.java index 649e2e6500506..47bfbd3d56709 100644 --- a/flink-python/src/main/java/org/apache/flink/python/PythonOptions.java +++ b/flink-python/src/main/java/org/apache/flink/python/PythonOptions.java @@ -7,7 +7,7 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -20,11 +20,14 @@ import org.apache.flink.annotation.Experimental; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.annotation.docs.Documentation; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.configuration.description.Description; +import java.util.Map; + /** Configuration options for the Python API. */ @PublicEvolving public class PythonOptions { @@ -231,10 +234,40 @@ public class PythonOptions { .stringType() .defaultValue("process") .withDescription( - "Specify the python runtime execution mode. The optional values are `process`, `multi-thread` and `sub-interpreter`. " + "Specify the python runtime execution mode. The optional values are `process` and `thread`. " + "The `process` mode means that the Python user-defined functions will be executed in separate Python process. " - + "The `multi-thread` mode means that the Python user-defined functions will be executed in the same thread as Java Operator, but it will be affected by GIL performance. " - + "The `sub-interpreter` mode means that the Python user-defined functions will be executed in python different sub-interpreters rather than different threads of one interpreter, " - + "which can largely overcome the effects of the GIL, but it maybe fail in some CPython extensions libraries, such as numpy, tensorflow. " - + "Note that if the python operator dose not support `multi-thread` and `sub-interpreter` mode, we will still use `process` mode."); + + "The `thread` mode means that the Python user-defined functions will be executed in the same process of the Java operator. " + + "Note that currently it still doesn't support to execute Python user-defined functions in `thread` mode in all places. " + + "It will fall back to `process` mode in these cases."); + + // ------------------------------------------------------------------------------------------ + // config options used for internal purpose + // ------------------------------------------------------------------------------------------ + + @Documentation.ExcludeFromDocumentation( + "Internal use only. The options will be exported as environment variables which could be accessed in Python worker process.") + public static final ConfigOption> PYTHON_JOB_OPTIONS = + ConfigOptions.key("python.job-options").mapType().noDefaultValue(); + + @Documentation.ExcludeFromDocumentation( + "Internal use only. The distributed cache entries for 'python.files'.") + public static final ConfigOption> PYTHON_FILES_DISTRIBUTED_CACHE_INFO = + ConfigOptions.key("python.internal.files-key-map").mapType().noDefaultValue(); + + @Documentation.ExcludeFromDocumentation( + "Internal use only. The distributed cache entries for 'python.requirements'.") + public static final ConfigOption> + PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO = + ConfigOptions.key("python.internal.requirements-file-key") + .mapType() + .noDefaultValue(); + + @Documentation.ExcludeFromDocumentation( + "Internal use only. The distributed cache entries for 'python.archives'.") + public static final ConfigOption> PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO = + ConfigOptions.key("python.internal.archives-key-map").mapType().noDefaultValue(); + + @Documentation.ExcludeFromDocumentation("Internal use only. Used for local debug.") + public static final ConfigOption PYTHON_LOOPBACK_SERVER_ADDRESS = + ConfigOptions.key("python.loopback-server.address").stringType().noDefaultValue(); } diff --git a/flink-python/src/main/java/org/apache/flink/python/env/PythonDependencyInfo.java b/flink-python/src/main/java/org/apache/flink/python/env/PythonDependencyInfo.java index 4ef685203b537..80f570b271491 100644 --- a/flink-python/src/main/java/org/apache/flink/python/env/PythonDependencyInfo.java +++ b/flink-python/src/main/java/org/apache/flink/python/env/PythonDependencyInfo.java @@ -20,8 +20,8 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.cache.DistributedCache; -import org.apache.flink.python.PythonConfig; -import org.apache.flink.python.PythonOptions; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.python.util.PythonDependencyUtils; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -33,6 +33,12 @@ import java.util.Objects; import java.util.Optional; +import static org.apache.flink.python.PythonOptions.PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO; +import static org.apache.flink.python.PythonOptions.PYTHON_EXECUTABLE; +import static org.apache.flink.python.PythonOptions.PYTHON_EXECUTION_MODE; +import static org.apache.flink.python.PythonOptions.PYTHON_FILES_DISTRIBUTED_CACHE_INFO; +import static org.apache.flink.python.PythonOptions.PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO; + /** PythonDependencyInfo contains the information of third-party dependencies. */ @Internal public final class PythonDependencyInfo { @@ -85,7 +91,7 @@ public PythonDependencyInfo( requirementsCacheDir, archives, pythonExec, - PythonOptions.PYTHON_EXECUTION_MODE.defaultValue()); + PYTHON_EXECUTION_MODE.defaultValue()); } public PythonDependencyInfo( @@ -130,15 +136,18 @@ public String getExecutionMode() { /** * Creates PythonDependencyInfo from GlobalJobParameters and DistributedCache. * - * @param pythonConfig The python config. + * @param config The config. * @param distributedCache The DistributedCache object of current task. * @return The PythonDependencyInfo object that contains whole information of python dependency. */ public static PythonDependencyInfo create( - PythonConfig pythonConfig, DistributedCache distributedCache) { + ReadableConfig config, DistributedCache distributedCache) { Map pythonFiles = new LinkedHashMap<>(); - for (Map.Entry entry : pythonConfig.getPythonFilesInfo().entrySet()) { + for (Map.Entry entry : + config.getOptional(PYTHON_FILES_DISTRIBUTED_CACHE_INFO) + .orElse(new HashMap<>()) + .entrySet()) { File pythonFile = distributedCache.getFile(entry.getKey()); String filePath = pythonFile.getAbsolutePath(); pythonFiles.put(filePath, entry.getValue()); @@ -146,27 +155,34 @@ public static PythonDependencyInfo create( String requirementsFilePath = null; String requirementsCacheDir = null; - if (pythonConfig.getPythonRequirementsFileInfo().isPresent()) { - requirementsFilePath = - distributedCache - .getFile(pythonConfig.getPythonRequirementsFileInfo().get()) - .getAbsolutePath(); - if (pythonConfig.getPythonRequirementsCacheDirInfo().isPresent()) { + + String requirementsFileName = + config.getOptional(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO) + .orElse(new HashMap<>()) + .get(PythonDependencyUtils.FILE); + if (requirementsFileName != null) { + requirementsFilePath = distributedCache.getFile(requirementsFileName).getAbsolutePath(); + String requirementsFileCacheDir = + config.getOptional(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO) + .orElse(new HashMap<>()) + .get(PythonDependencyUtils.CACHE); + if (requirementsFileCacheDir != null) { requirementsCacheDir = - distributedCache - .getFile(pythonConfig.getPythonRequirementsCacheDirInfo().get()) - .getAbsolutePath(); + distributedCache.getFile(requirementsFileCacheDir).getAbsolutePath(); } } Map archives = new HashMap<>(); - for (Map.Entry entry : pythonConfig.getPythonArchivesInfo().entrySet()) { + for (Map.Entry entry : + config.getOptional(PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO) + .orElse(new HashMap<>()) + .entrySet()) { String archiveFilePath = distributedCache.getFile(entry.getKey()).getAbsolutePath(); String targetPath = entry.getValue(); archives.put(archiveFilePath, targetPath); } - String pythonExec = pythonConfig.getPythonExec(); + String pythonExec = config.get(PYTHON_EXECUTABLE); return new PythonDependencyInfo( pythonFiles, @@ -174,6 +190,6 @@ public static PythonDependencyInfo create( requirementsCacheDir, archives, pythonExec, - pythonConfig.getExecutionMode()); + config.get(PYTHON_EXECUTION_MODE)); } } diff --git a/flink-python/src/main/java/org/apache/flink/python/env/embedded/EmbeddedPythonEnvironmentManager.java b/flink-python/src/main/java/org/apache/flink/python/env/embedded/EmbeddedPythonEnvironmentManager.java index 1d7e9a037b7dc..750284e2edfe4 100644 --- a/flink-python/src/main/java/org/apache/flink/python/env/embedded/EmbeddedPythonEnvironmentManager.java +++ b/flink-python/src/main/java/org/apache/flink/python/env/embedded/EmbeddedPythonEnvironmentManager.java @@ -54,9 +54,7 @@ public PythonEnvironment createEnvironment() throws Exception { String executionMode = dependencyInfo.getExecutionMode(); - if (executionMode.equalsIgnoreCase("sub-interpreter")) { - execType = PythonInterpreterConfig.ExecType.SUB_INTERPRETER; - } else if (executionMode.equalsIgnoreCase("multi-thread")) { + if (executionMode.equalsIgnoreCase("thread")) { execType = PythonInterpreterConfig.ExecType.MULTI_THREAD; } else { throw new RuntimeException( @@ -66,16 +64,8 @@ public PythonEnvironment createEnvironment() throws Exception { String pythonVersion = PythonEnvironmentManagerUtils.getPythonVersion(dependencyInfo.getPythonExec()); - if (execType == PythonInterpreterConfig.ExecType.SUB_INTERPRETER) { - if (pythonVersion.compareTo("3.8") < 0) { - throw new RuntimeException( - "`SUB-INTERPRETER` execution mode only supports Python 3.8+"); - } - } else { - if (pythonVersion.compareTo("3.7") < 0) { - throw new RuntimeException( - "`MULTI-THREAD` execution mode only supports Python 3.7+"); - } + if (pythonVersion.compareTo("3.7") < 0) { + throw new RuntimeException("`THREAD` execution mode only supports Python 3.7+"); } if (env.containsKey("FLINK_TESTING")) { diff --git a/flink-python/src/main/java/org/apache/flink/python/metric/FlinkMetricContainer.java b/flink-python/src/main/java/org/apache/flink/python/metric/FlinkMetricContainer.java index 96e127faf2782..a184c39c6211e 100644 --- a/flink-python/src/main/java/org/apache/flink/python/metric/FlinkMetricContainer.java +++ b/flink-python/src/main/java/org/apache/flink/python/metric/FlinkMetricContainer.java @@ -33,7 +33,6 @@ import org.apache.beam.model.pipeline.v1.MetricsApi.MonitoringInfo; import org.apache.beam.runners.core.metrics.MetricsContainerImpl; import org.apache.beam.runners.core.metrics.MetricsContainerStepMap; -import org.apache.beam.runners.core.metrics.MonitoringInfoConstants; import org.apache.beam.runners.core.metrics.MonitoringInfoMetricName; import org.apache.beam.sdk.metrics.DistributionResult; import org.apache.beam.sdk.metrics.GaugeResult; @@ -106,10 +105,7 @@ private boolean isUserMetric(MetricResult metricResult) { MetricName metricName = metricResult.getKey().metricName(); if (metricName instanceof MonitoringInfoMetricName) { String urn = ((MonitoringInfoMetricName) metricName).getUrn(); - return urn.contains(MonitoringInfoConstants.Urns.USER_SUM_INT64) - || urn.contains(MonitoringInfoConstants.Urns.USER_SUM_DOUBLE) - || urn.contains(MonitoringInfoConstants.Urns.USER_DISTRIBUTION_DOUBLE) - || urn.contains(MonitoringInfoConstants.Urns.USER_DISTRIBUTION_INT64); + return urn.startsWith("beam:metric:user"); } return false; } diff --git a/flink-python/src/main/java/org/apache/flink/python/util/PythonConfigUtil.java b/flink-python/src/main/java/org/apache/flink/python/util/PythonConfigUtil.java index 18ddb5845c085..3c6085b725475 100644 --- a/flink-python/src/main/java/org/apache/flink/python/util/PythonConfigUtil.java +++ b/flink-python/src/main/java/org/apache/flink/python/util/PythonConfigUtil.java @@ -19,10 +19,11 @@ import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.api.dag.Transformation; -import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; +import org.apache.flink.python.PythonConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.streaming.api.operators.StreamOperatorFactory; @@ -34,8 +35,6 @@ import org.apache.flink.streaming.api.transformations.PartitionTransformation; import org.apache.flink.streaming.api.transformations.TwoInputTransformation; import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner; -import org.apache.flink.table.api.TableConfig; -import org.apache.flink.table.api.TableException; import org.apache.flink.shaded.guava30.com.google.common.collect.Queues; import org.apache.flink.shaded.guava30.com.google.common.collect.Sets; @@ -47,10 +46,7 @@ import java.util.Queue; import java.util.Set; -/** - * A Util class to get the {@link StreamExecutionEnvironment} configuration and merged configuration - * with environment settings. - */ +/** A Util class to handle the configurations of Python jobs. */ public class PythonConfigUtil { public static final String KEYED_STREAM_VALUE_OPERATOR_NAME = "_keyed_stream_values_operator"; @@ -59,19 +55,9 @@ public class PythonConfigUtil { "_partition_custom_map_operator"; /** - * A static method to get the {@link StreamExecutionEnvironment} configuration merged with - * python dependency management configurations. - */ - public static Configuration getEnvConfigWithDependencies(StreamExecutionEnvironment env) - throws InvocationTargetException, IllegalAccessException, NoSuchFieldException { - return PythonDependencyUtils.configurePythonDependencies( - env.getCachedFiles(), (Configuration) env.getConfiguration()); - } - - /** - * Get the private field {@code StreamExecutionEnvironment#configuration} by reflection - * recursively. Then access the field to get the configuration of the given - * StreamExecutionEnvironment. + * Get the private field {@link StreamExecutionEnvironment#configuration} by reflection + * recursively. It allows modification to the configuration compared with {@link + * StreamExecutionEnvironment#getConfiguration()}. */ public static Configuration getEnvironmentConfig(StreamExecutionEnvironment env) throws InvocationTargetException, IllegalAccessException, NoSuchFieldException { @@ -93,62 +79,35 @@ public static Configuration getEnvironmentConfig(StreamExecutionEnvironment env) return (Configuration) configurationField.get(env); } - @SuppressWarnings("unchecked") public static void configPythonOperator(StreamExecutionEnvironment env) - throws IllegalAccessException, InvocationTargetException, NoSuchFieldException { - Configuration mergedConfig = getEnvConfigWithDependencies(env); + throws IllegalAccessException, NoSuchFieldException { + final Configuration config = + extractPythonConfiguration(env.getCachedFiles(), env.getConfiguration()); - Field transformationsField = - StreamExecutionEnvironment.class.getDeclaredField("transformations"); - transformationsField.setAccessible(true); - List> transformations = - (List>) transformationsField.get(env); - for (Transformation transformation : transformations) { + for (Transformation transformation : env.getTransformations()) { alignTransformation(transformation); if (isPythonOperator(transformation)) { - // declare it is a Python operator + // declare the use case of managed memory transformation.declareManagedMemoryUseCaseAtSlotScope(ManagedMemoryUseCase.PYTHON); AbstractPythonFunctionOperator pythonFunctionOperator = getPythonOperator(transformation); if (pythonFunctionOperator != null) { - Configuration oldConfig = pythonFunctionOperator.getConfiguration(); - // update dependency related configurations for Python operators - pythonFunctionOperator.setConfiguration( - generateNewPythonConfig(oldConfig, mergedConfig)); + pythonFunctionOperator.getConfiguration().addAll(config); } } } } - public static Configuration getMergedConfig( - StreamExecutionEnvironment env, TableConfig tableConfig) { - Configuration config = new Configuration((Configuration) env.getConfiguration()); - PythonDependencyUtils.merge(config, tableConfig.getConfiguration()); - Configuration mergedConfig = - PythonDependencyUtils.configurePythonDependencies(env.getCachedFiles(), config); - mergedConfig.setString("table.exec.timezone", tableConfig.getLocalTimeZone().getId()); - return mergedConfig; - } - - @SuppressWarnings("unchecked") - public static Configuration getMergedConfig(ExecutionEnvironment env, TableConfig tableConfig) { - try { - Field field = ExecutionEnvironment.class.getDeclaredField("cacheFile"); - field.setAccessible(true); - Configuration config = new Configuration(env.getConfiguration()); - PythonDependencyUtils.merge(config, tableConfig.getConfiguration()); - Configuration mergedConfig = - PythonDependencyUtils.configurePythonDependencies( - (List>) - field.get(env), - config); - mergedConfig.setString("table.exec.timezone", tableConfig.getLocalTimeZone().getId()); - return mergedConfig; - } catch (NoSuchFieldException | IllegalAccessException e) { - throw new TableException("Method getMergedConfig failed.", e); - } + /** Extract the configurations which is used in the Python operators. */ + public static Configuration extractPythonConfiguration( + List> cachedFiles, + ReadableConfig config) { + final Configuration pythonDependencyConfig = + PythonDependencyUtils.configurePythonDependencies(cachedFiles, config); + final PythonConfig pythonConfig = new PythonConfig(config, pythonDependencyConfig); + return pythonConfig.toConfiguration(); } /** @@ -220,7 +179,7 @@ private static AbstractPythonFunctionOperator getPythonOperator( return null; } - public static boolean isPythonOperator(Transformation transform) { + private static boolean isPythonOperator(Transformation transform) { if (transform instanceof OneInputTransformation) { return isPythonOperator( ((OneInputTransformation) transform).getOperatorFactory()); @@ -266,17 +225,6 @@ private static boolean isPythonDataStreamOperator( } } - /** - * Generator a new {@link Configuration} with the combined config which is derived from - * oldConfig. - */ - private static Configuration generateNewPythonConfig( - Configuration oldConfig, Configuration newConfig) { - Configuration mergedConfig = newConfig.clone(); - mergedConfig.addAll(oldConfig); - return mergedConfig; - } - public static void setPartitionCustomOperatorNumPartitions( List> transformations) { // Update the numPartitions of PartitionCustomOperator after aligned all operators. diff --git a/flink-python/src/main/java/org/apache/flink/python/util/PythonDependencyUtils.java b/flink-python/src/main/java/org/apache/flink/python/util/PythonDependencyUtils.java index f90aa72f8c333..c93a6f4994901 100644 --- a/flink-python/src/main/java/org/apache/flink/python/util/PythonDependencyUtils.java +++ b/flink-python/src/main/java/org/apache/flink/python/util/PythonDependencyUtils.java @@ -20,8 +20,6 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.python.PythonOptions; @@ -39,7 +37,6 @@ import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; import static org.apache.flink.client.cli.CliFrontendParser.PYARCHIVE_OPTION; @@ -47,8 +44,11 @@ import static org.apache.flink.client.cli.CliFrontendParser.PYEXEC_OPTION; import static org.apache.flink.client.cli.CliFrontendParser.PYFILES_OPTION; import static org.apache.flink.client.cli.CliFrontendParser.PYREQUIREMENTS_OPTION; +import static org.apache.flink.python.PythonOptions.PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO; import static org.apache.flink.python.PythonOptions.PYTHON_CLIENT_EXECUTABLE; import static org.apache.flink.python.PythonOptions.PYTHON_EXECUTABLE; +import static org.apache.flink.python.PythonOptions.PYTHON_FILES_DISTRIBUTED_CACHE_INFO; +import static org.apache.flink.python.PythonOptions.PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO; /** * Utility class for Python dependency management. The dependencies will be registered at the @@ -63,15 +63,6 @@ public class PythonDependencyUtils { public static final String PARAM_DELIMITER = "#"; private static final String HASH_ALGORITHM = "SHA-256"; - // Internal Python Config Options. - - public static final ConfigOption> PYTHON_FILES = - ConfigOptions.key("python.internal.files-key-map").mapType().noDefaultValue(); - public static final ConfigOption> PYTHON_REQUIREMENTS_FILE = - ConfigOptions.key("python.internal.requirements-file-key").mapType().noDefaultValue(); - public static final ConfigOption> PYTHON_ARCHIVES = - ConfigOptions.key("python.internal.archives-key-map").mapType().noDefaultValue(); - /** * Adds python dependencies to registered cache file list according to given configuration and * returns a new configuration which contains the metadata of the registered python @@ -84,10 +75,12 @@ public class PythonDependencyUtils { */ public static Configuration configurePythonDependencies( List> cachedFiles, - Configuration config) { - PythonDependencyManager pythonDependencyManager = + ReadableConfig config) { + final PythonDependencyManager pythonDependencyManager = new PythonDependencyManager(cachedFiles, config); - return pythonDependencyManager.getConfigWithPythonDependencyOptions(); + final Configuration pythonDependencyConfig = new Configuration(); + pythonDependencyManager.applyToConfiguration(pythonDependencyConfig); + return pythonDependencyConfig; } public static Configuration parsePythonDependencyConfiguration(CommandLine commandLine) { @@ -162,14 +155,13 @@ private static class PythonDependencyManager { private static final String PYTHON_ARCHIVE_PREFIX = "python_archive"; private final List> cachedFiles; - private final Configuration internalConfig; + private final ReadableConfig config; private PythonDependencyManager( List> cachedFiles, - Configuration config) { + ReadableConfig config) { this.cachedFiles = cachedFiles; - this.internalConfig = new Configuration(config); - configure(config); + this.config = config; } /** @@ -179,14 +171,17 @@ private PythonDependencyManager( * * @param filePath The path of the Python dependency. */ - private void addPythonFile(String filePath) { + private void addPythonFile(Configuration pythonDependencyConfig, String filePath) { Preconditions.checkNotNull(filePath); String fileKey = generateUniqueFileKey(PYTHON_FILE_PREFIX, filePath); registerCachedFileIfNotExist(filePath, fileKey); - if (!internalConfig.contains(PYTHON_FILES)) { - internalConfig.set(PYTHON_FILES, new LinkedHashMap<>()); + if (!pythonDependencyConfig.contains(PYTHON_FILES_DISTRIBUTED_CACHE_INFO)) { + pythonDependencyConfig.set( + PYTHON_FILES_DISTRIBUTED_CACHE_INFO, new LinkedHashMap<>()); } - internalConfig.get(PYTHON_FILES).put(fileKey, new File(filePath).getName()); + pythonDependencyConfig + .get(PYTHON_FILES_DISTRIBUTED_CACHE_INFO) + .put(fileKey, new File(filePath).getName()); } /** @@ -196,8 +191,9 @@ private void addPythonFile(String filePath) { * * @param requirementsFilePath The path of the requirements file. */ - private void setPythonRequirements(String requirementsFilePath) { - setPythonRequirements(requirementsFilePath, null); + private void setPythonRequirements( + Configuration pythonDependencyConfig, String requirementsFilePath) { + setPythonRequirements(pythonDependencyConfig, requirementsFilePath, null); } /** @@ -210,26 +206,33 @@ private void setPythonRequirements(String requirementsFilePath) { * @param requirementsCachedDir The path of the requirements cached directory. */ private void setPythonRequirements( - String requirementsFilePath, @Nullable String requirementsCachedDir) { + Configuration pythonDependencyConfig, + String requirementsFilePath, + @Nullable String requirementsCachedDir) { Preconditions.checkNotNull(requirementsFilePath); - if (!internalConfig.contains(PYTHON_REQUIREMENTS_FILE)) { - internalConfig.set(PYTHON_REQUIREMENTS_FILE, new HashMap<>()); + if (!pythonDependencyConfig.contains(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO)) { + pythonDependencyConfig.set( + PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO, new HashMap<>()); } - internalConfig.get(PYTHON_REQUIREMENTS_FILE).clear(); + pythonDependencyConfig.get(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO).clear(); removeCachedFilesByPrefix(PYTHON_REQUIREMENTS_FILE_PREFIX); removeCachedFilesByPrefix(PYTHON_REQUIREMENTS_CACHE_PREFIX); String fileKey = generateUniqueFileKey(PYTHON_REQUIREMENTS_FILE_PREFIX, requirementsFilePath); registerCachedFileIfNotExist(requirementsFilePath, fileKey); - internalConfig.get(PYTHON_REQUIREMENTS_FILE).put(FILE, fileKey); + pythonDependencyConfig + .get(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO) + .put(FILE, fileKey); if (requirementsCachedDir != null) { String cacheDirKey = generateUniqueFileKey( PYTHON_REQUIREMENTS_CACHE_PREFIX, requirementsCachedDir); registerCachedFileIfNotExist(requirementsCachedDir, cacheDirKey); - internalConfig.get(PYTHON_REQUIREMENTS_FILE).put(CACHE, cacheDirKey); + pythonDependencyConfig + .get(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO) + .put(CACHE, cacheDirKey); } } @@ -242,24 +245,27 @@ private void setPythonRequirements( * @param archivePath The path of the archive file. * @param targetDir The name of the target directory. */ - private void addPythonArchive(String archivePath, String targetDir) { + private void addPythonArchive( + Configuration pythonDependencyConfig, String archivePath, String targetDir) { Preconditions.checkNotNull(archivePath); - if (!internalConfig.contains(PYTHON_ARCHIVES)) { - internalConfig.set(PYTHON_ARCHIVES, new HashMap<>()); + if (!pythonDependencyConfig.contains(PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO)) { + pythonDependencyConfig.set(PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO, new HashMap<>()); } String fileKey = generateUniqueFileKey( PYTHON_ARCHIVE_PREFIX, archivePath + PARAM_DELIMITER + targetDir); registerCachedFileIfNotExist(archivePath, fileKey); - internalConfig.get(PYTHON_ARCHIVES).put(fileKey, targetDir); + pythonDependencyConfig + .get(PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO) + .put(fileKey, targetDir); } - private void configure(ReadableConfig config) { + private void applyToConfiguration(Configuration pythonDependencyConfig) { config.getOptional(PythonOptions.PYTHON_FILES) .ifPresent( pyFiles -> { for (String filePath : pyFiles.split(FILE_DELIMITER)) { - addPythonFile(filePath); + addPythonFile(pythonDependencyConfig, filePath); } }); @@ -270,9 +276,11 @@ private void configure(ReadableConfig config) { String[] requirementFileAndCache = pyRequirements.split(PARAM_DELIMITER, 2); setPythonRequirements( - requirementFileAndCache[0], requirementFileAndCache[1]); + pythonDependencyConfig, + requirementFileAndCache[0], + requirementFileAndCache[1]); } else { - setPythonRequirements(pyRequirements); + setPythonRequirements(pythonDependencyConfig, pyRequirements); } }); @@ -294,15 +302,16 @@ private void configure(ReadableConfig config) { archivePath = archive; targetDir = new File(archivePath).getName(); } - addPythonArchive(archivePath, targetDir); + addPythonArchive( + pythonDependencyConfig, archivePath, targetDir); } }); config.getOptional(PYTHON_EXECUTABLE) - .ifPresent(e -> internalConfig.set(PYTHON_EXECUTABLE, e)); + .ifPresent(e -> pythonDependencyConfig.set(PYTHON_EXECUTABLE, e)); config.getOptional(PYTHON_CLIENT_EXECUTABLE) - .ifPresent(e -> internalConfig.set(PYTHON_CLIENT_EXECUTABLE, e)); + .ifPresent(e -> pythonDependencyConfig.set(PYTHON_CLIENT_EXECUTABLE, e)); } private String generateUniqueFileKey(String prefix, String hashString) { @@ -333,9 +342,5 @@ private void removeCachedFilesByPrefix(String prefix) { .filter(t -> t.f0.matches("^" + prefix + "_[a-z0-9]{64}$")) .collect(Collectors.toSet())); } - - private Configuration getConfigWithPythonDependencyOptions() { - return internalConfig; - } } } diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractEmbeddedPythonFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractEmbeddedPythonFunctionOperator.java index f23e3d2baf814..6dc961832ee4d 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractEmbeddedPythonFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractEmbeddedPythonFunctionOperator.java @@ -23,7 +23,6 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.fnexecution.v1.FlinkFnApi; -import org.apache.flink.python.PythonConfig; import org.apache.flink.python.env.PythonDependencyInfo; import org.apache.flink.python.env.embedded.EmbeddedPythonEnvironment; import org.apache.flink.python.env.embedded.EmbeddedPythonEnvironmentManager; @@ -36,6 +35,7 @@ import java.util.Map; import java.util.concurrent.locks.ReentrantLock; +import static org.apache.flink.python.PythonOptions.PYTHON_EXECUTABLE; import static org.apache.flink.python.env.AbstractPythonEnvironmentManager.PYTHON_WORKING_DIR; /** @@ -48,12 +48,9 @@ public abstract class AbstractEmbeddedPythonFunctionOperator private static final long serialVersionUID = 1L; - private static ReentrantLock lock = new ReentrantLock(); + private static final ReentrantLock lock = new ReentrantLock(); - private static Map> workingDirectories = new HashMap<>(); - - /** The python config. */ - protected transient PythonConfig pythonConfig; + private static final Map> workingDirectories = new HashMap<>(); /** Every operator will hold the only python interpreter. */ protected transient PythonInterpreter interpreter; @@ -67,7 +64,6 @@ public AbstractEmbeddedPythonFunctionOperator(Configuration config) { @Override public void open() throws Exception { super.open(); - pythonConfig = new PythonConfig(config); pythonEnvironmentManager = createPythonEnvironmentManager(); pythonEnvironmentManager.open(); EmbeddedPythonEnvironment environment = @@ -105,7 +101,7 @@ public void open() throws Exception { } } - openPythonInterpreter(pythonConfig.getPythonExec(), env, interpreterConfig.getExecType()); + openPythonInterpreter(config.get(PYTHON_EXECUTABLE), env); } @Override @@ -142,8 +138,7 @@ public void close() throws Exception { @Override protected EmbeddedPythonEnvironmentManager createPythonEnvironmentManager() { PythonDependencyInfo dependencyInfo = - PythonDependencyInfo.create( - pythonConfig, getRuntimeContext().getDistributedCache()); + PythonDependencyInfo.create(config, getRuntimeContext().getDistributedCache()); return new EmbeddedPythonEnvironmentManager( dependencyInfo, getContainingTask().getEnvironment().getTaskManagerInfo().getTmpDirectories(), @@ -152,11 +147,7 @@ protected EmbeddedPythonEnvironmentManager createPythonEnvironmentManager() { } /** Setup method for Python Interpreter. It can be used for initialization work. */ - public abstract void openPythonInterpreter( - String pythonExecutable, - Map env, - PythonInterpreterConfig.ExecType execType) - throws Exception; + public abstract void openPythonInterpreter(String pythonExecutable, Map env); /** Returns the {@link PythonEnv} used to create PythonEnvironmentManager. */ public abstract PythonEnv getPythonEnv(); diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractExternalPythonFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractExternalPythonFunctionOperator.java index 702e42e06973b..d95ffafb5afff 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractExternalPythonFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractExternalPythonFunctionOperator.java @@ -53,7 +53,7 @@ public AbstractExternalPythonFunctionOperator(Configuration config) { public void open() throws Exception { super.open(); this.pythonFunctionRunner = createPythonFunctionRunner(); - this.pythonFunctionRunner.open(pythonConfig); + this.pythonFunctionRunner.open(config); this.flushThreadPool = Executors.newSingleThreadExecutor(); } @@ -120,8 +120,7 @@ protected void invokeFinishBundle() throws Exception { @Override protected ProcessPythonEnvironmentManager createPythonEnvironmentManager() { PythonDependencyInfo dependencyInfo = - PythonDependencyInfo.create( - pythonConfig, getRuntimeContext().getDistributedCache()); + PythonDependencyInfo.create(config, getRuntimeContext().getDistributedCache()); PythonEnv pythonEnv = getPythonEnv(); if (pythonEnv.getExecType() == PythonEnv.ExecType.PROCESS) { return new ProcessPythonEnvironmentManager( diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractPythonFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractPythonFunctionOperator.java index aa1dfe1792093..9286b786506d4 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractPythonFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractPythonFunctionOperator.java @@ -20,8 +20,6 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.configuration.Configuration; -import org.apache.flink.python.PythonConfig; -import org.apache.flink.python.PythonOptions; import org.apache.flink.python.env.PythonEnvironmentManager; import org.apache.flink.python.metric.FlinkMetricContainer; import org.apache.flink.runtime.state.KeyedStateBackend; @@ -36,10 +34,12 @@ import java.lang.reflect.Field; import java.util.List; -import java.util.Map; import java.util.Objects; import java.util.concurrent.ScheduledFuture; +import static org.apache.flink.python.PythonOptions.MAX_BUNDLE_SIZE; +import static org.apache.flink.python.PythonOptions.MAX_BUNDLE_TIME_MILLS; +import static org.apache.flink.python.PythonOptions.PYTHON_METRIC_ENABLED; import static org.apache.flink.streaming.api.utils.ClassLeakCleaner.cleanUpLeakingClasses; import static org.apache.flink.streaming.api.utils.PythonOperatorUtils.inBatchExecutionMode; @@ -49,7 +49,7 @@ public abstract class AbstractPythonFunctionOperator extends AbstractStream private static final long serialVersionUID = 1L; - protected Configuration config; + protected final Configuration config; /** Max number of elements to include in a bundle. */ protected transient int maxBundleSize; @@ -57,12 +57,6 @@ public abstract class AbstractPythonFunctionOperator extends AbstractStream /** Number of processed elements in the current bundle. */ protected transient int elementCount; - /** The python config. */ - protected transient PythonConfig pythonConfig; - - /** The options used to configure the Python worker process. */ - protected transient Map jobOptions; - /** Max duration of a bundle. */ private transient long maxBundleTimeMills; @@ -83,12 +77,10 @@ public AbstractPythonFunctionOperator(Configuration config) { @Override public void open() throws Exception { try { - this.pythonConfig = new PythonConfig(config); - this.jobOptions = config.toMap(); - this.maxBundleSize = pythonConfig.getMaxBundleSize(); + this.maxBundleSize = config.get(MAX_BUNDLE_SIZE); if (this.maxBundleSize <= 0) { - this.maxBundleSize = PythonOptions.MAX_BUNDLE_SIZE.defaultValue(); - LOG.error( + this.maxBundleSize = MAX_BUNDLE_SIZE.defaultValue(); + LOG.warn( "Invalid value for the maximum bundle size. Using default value of " + this.maxBundleSize + '.'); @@ -96,10 +88,10 @@ public void open() throws Exception { LOG.info("The maximum bundle size is configured to {}.", this.maxBundleSize); } - this.maxBundleTimeMills = pythonConfig.getMaxBundleTimeMills(); + this.maxBundleTimeMills = config.get(MAX_BUNDLE_TIME_MILLS); if (this.maxBundleTimeMills <= 0L) { - this.maxBundleTimeMills = PythonOptions.MAX_BUNDLE_TIME_MILLS.defaultValue(); - LOG.error( + this.maxBundleTimeMills = MAX_BUNDLE_TIME_MILLS.defaultValue(); + LOG.warn( "Invalid value for the maximum bundle time. Using default value of " + this.maxBundleTimeMills + '.'); @@ -256,11 +248,6 @@ public boolean isBundleFinished() { return elementCount == 0; } - /** Reset the {@link Configuration} if needed. */ - public void setConfiguration(Configuration config) { - this.config = config; - } - /** Returns the {@link Configuration}. */ public Configuration getConfiguration() { return config; @@ -289,7 +276,7 @@ private void checkInvokeFinishBundleByTime() throws Exception { } protected FlinkMetricContainer getFlinkMetricContainer() { - return this.pythonConfig.isMetricEnabled() + return this.config.get(PYTHON_METRIC_ENABLED) ? new FlinkMetricContainer(getRuntimeContext().getMetricGroup()) : null; } diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonCoProcessOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonCoProcessOperator.java index 820860014c7a3..3a13f6725dda2 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonCoProcessOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonCoProcessOperator.java @@ -30,6 +30,11 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import static org.apache.flink.python.Constants.STATELESS_FUNCTION_URN; +import static org.apache.flink.python.PythonOptions.MAP_STATE_READ_CACHE_SIZE; +import static org.apache.flink.python.PythonOptions.MAP_STATE_WRITE_CACHE_SIZE; +import static org.apache.flink.python.PythonOptions.PYTHON_METRIC_ENABLED; +import static org.apache.flink.python.PythonOptions.PYTHON_PROFILE_ENABLED; +import static org.apache.flink.python.PythonOptions.STATE_CACHE_SIZE; import static org.apache.flink.streaming.api.utils.PythonOperatorUtils.inBatchExecutionMode; /** @@ -73,8 +78,12 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { getPythonFunctionInfo(), getRuntimeContext(), getInternalParameters(), - inBatchExecutionMode(getKeyedStateBackend())), - jobOptions, + inBatchExecutionMode(getKeyedStateBackend()), + config.get(PYTHON_METRIC_ENABLED), + config.get(PYTHON_PROFILE_ENABLED), + config.get(STATE_CACHE_SIZE), + config.get(MAP_STATE_READ_CACHE_SIZE), + config.get(MAP_STATE_WRITE_CACHE_SIZE)), getFlinkMetricContainer(), null, null, diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedCoProcessOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedCoProcessOperator.java index ba1c94c1a3f22..26ccaedf9ad48 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedCoProcessOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedCoProcessOperator.java @@ -41,6 +41,11 @@ import org.apache.flink.types.Row; import static org.apache.flink.python.Constants.STATEFUL_FUNCTION_URN; +import static org.apache.flink.python.PythonOptions.MAP_STATE_READ_CACHE_SIZE; +import static org.apache.flink.python.PythonOptions.MAP_STATE_WRITE_CACHE_SIZE; +import static org.apache.flink.python.PythonOptions.PYTHON_METRIC_ENABLED; +import static org.apache.flink.python.PythonOptions.PYTHON_PROFILE_ENABLED; +import static org.apache.flink.python.PythonOptions.STATE_CACHE_SIZE; import static org.apache.flink.streaming.api.operators.python.timer.TimerUtils.createTimerDataCoderInfoDescriptorProto; import static org.apache.flink.streaming.api.operators.python.timer.TimerUtils.createTimerDataTypeInfo; import static org.apache.flink.streaming.api.utils.PythonOperatorUtils.inBatchExecutionMode; @@ -112,8 +117,12 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { getRuntimeContext(), getInternalParameters(), keyTypeInfo, - inBatchExecutionMode(getKeyedStateBackend())), - jobOptions, + inBatchExecutionMode(getKeyedStateBackend()), + config.get(PYTHON_METRIC_ENABLED), + config.get(PYTHON_PROFILE_ENABLED), + config.get(STATE_CACHE_SIZE), + config.get(MAP_STATE_READ_CACHE_SIZE), + config.get(MAP_STATE_WRITE_CACHE_SIZE)), getFlinkMetricContainer(), getKeyedStateBackend(), keyTypeSerializer, diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedProcessOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedProcessOperator.java index 3d81c1076bac4..2896ed3be4731 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedProcessOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedProcessOperator.java @@ -41,6 +41,11 @@ import org.apache.flink.types.Row; import static org.apache.flink.python.Constants.STATEFUL_FUNCTION_URN; +import static org.apache.flink.python.PythonOptions.MAP_STATE_READ_CACHE_SIZE; +import static org.apache.flink.python.PythonOptions.MAP_STATE_WRITE_CACHE_SIZE; +import static org.apache.flink.python.PythonOptions.PYTHON_METRIC_ENABLED; +import static org.apache.flink.python.PythonOptions.PYTHON_PROFILE_ENABLED; +import static org.apache.flink.python.PythonOptions.STATE_CACHE_SIZE; import static org.apache.flink.streaming.api.operators.python.timer.TimerUtils.createTimerDataCoderInfoDescriptorProto; import static org.apache.flink.streaming.api.operators.python.timer.TimerUtils.createTimerDataTypeInfo; import static org.apache.flink.streaming.api.utils.PythonOperatorUtils.inBatchExecutionMode; @@ -142,8 +147,12 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { getRuntimeContext(), getInternalParameters(), keyTypeInfo, - inBatchExecutionMode(getKeyedStateBackend())), - jobOptions, + inBatchExecutionMode(getKeyedStateBackend()), + config.get(PYTHON_METRIC_ENABLED), + config.get(PYTHON_PROFILE_ENABLED), + config.get(STATE_CACHE_SIZE), + config.get(MAP_STATE_READ_CACHE_SIZE), + config.get(MAP_STATE_WRITE_CACHE_SIZE)), getFlinkMetricContainer(), getKeyedStateBackend(), keyTypeSerializer, diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonProcessOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonProcessOperator.java index 69d4e74146b42..48e11fab123fa 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonProcessOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonProcessOperator.java @@ -31,6 +31,11 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import static org.apache.flink.python.Constants.STATELESS_FUNCTION_URN; +import static org.apache.flink.python.PythonOptions.MAP_STATE_READ_CACHE_SIZE; +import static org.apache.flink.python.PythonOptions.MAP_STATE_WRITE_CACHE_SIZE; +import static org.apache.flink.python.PythonOptions.PYTHON_METRIC_ENABLED; +import static org.apache.flink.python.PythonOptions.PYTHON_PROFILE_ENABLED; +import static org.apache.flink.python.PythonOptions.STATE_CACHE_SIZE; import static org.apache.flink.streaming.api.utils.PythonOperatorUtils.inBatchExecutionMode; /** @@ -70,8 +75,12 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { getPythonFunctionInfo(), getRuntimeContext(), getInternalParameters(), - inBatchExecutionMode(getKeyedStateBackend())), - jobOptions, + inBatchExecutionMode(getKeyedStateBackend()), + config.get(PYTHON_METRIC_ENABLED), + config.get(PYTHON_PROFILE_ENABLED), + config.get(STATE_CACHE_SIZE), + config.get(MAP_STATE_READ_CACHE_SIZE), + config.get(MAP_STATE_WRITE_CACHE_SIZE)), getFlinkMetricContainer(), null, null, diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/timer/TimerRegistration.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/timer/TimerRegistration.java index d554c696b9c8d..ccd80a55586a3 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/timer/TimerRegistration.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/timer/TimerRegistration.java @@ -27,6 +27,7 @@ import org.apache.flink.runtime.state.VoidNamespaceSerializer; import org.apache.flink.streaming.api.operators.InternalTimerService; import org.apache.flink.streaming.api.operators.KeyContext; +import org.apache.flink.streaming.api.utils.PythonOperatorUtils; import org.apache.flink.types.Row; import java.util.HashMap; @@ -83,9 +84,11 @@ public void setTimer(byte[] serializedTimerData) { } } - private void setTimer(TimerOperandType operandType, long timestamp, Row key, Object namespace) { + private void setTimer(TimerOperandType operandType, long timestamp, Row key, Object namespace) + throws Exception { synchronized (keyedStateBackend) { keyContext.setCurrentKey(key); + PythonOperatorUtils.setCurrentKeyForTimerService(internalTimerService, key); switch (operandType) { case REGISTER_EVENT_TIMER: internalTimerService.registerEventTimeTimer(namespace, timestamp); diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamDataStreamPythonFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamDataStreamPythonFunctionRunner.java index a76871f7acbd9..df3a651b8d14b 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamDataStreamPythonFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamDataStreamPythonFunctionRunner.java @@ -37,7 +37,6 @@ import java.util.Collections; import java.util.List; -import java.util.Map; import java.util.Optional; import static org.apache.flink.python.Constants.INPUT_COLLECTION_ID; @@ -71,11 +70,10 @@ public BeamDataStreamPythonFunctionRunner( ProcessPythonEnvironmentManager environmentManager, String headOperatorFunctionUrn, List userDefinedDataStreamFunctions, - Map jobOptions, @Nullable FlinkMetricContainer flinkMetricContainer, - KeyedStateBackend stateBackend, - TypeSerializer keySerializer, - TypeSerializer namespaceSerializer, + @Nullable KeyedStateBackend stateBackend, + @Nullable TypeSerializer keySerializer, + @Nullable TypeSerializer namespaceSerializer, @Nullable TimerRegistration timerRegistration, MemoryManager memoryManager, double managedMemoryFraction, @@ -85,7 +83,6 @@ public BeamDataStreamPythonFunctionRunner( super( taskName, environmentManager, - jobOptions, flinkMetricContainer, stateBackend, keySerializer, diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java index 86264ab90eaa6..ad04d9b5299bb 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java @@ -22,8 +22,8 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.fnexecution.v1.FlinkFnApi; -import org.apache.flink.python.PythonConfig; import org.apache.flink.python.PythonFunctionRunner; import org.apache.flink.python.PythonOptions; import org.apache.flink.python.env.PythonEnvironment; @@ -94,6 +94,7 @@ import static org.apache.flink.python.Constants.WINDOW_CODER_ID; import static org.apache.flink.python.Constants.WINDOW_STRATEGY; import static org.apache.flink.python.Constants.WRAPPER_TIMER_CODER_ID; +import static org.apache.flink.python.PythonOptions.USE_MANAGED_MEMORY; import static org.apache.flink.streaming.api.utils.ProtoUtils.createCoderProto; /** A {@link BeamPythonFunctionRunner} used to execute Python functions. */ @@ -112,13 +113,16 @@ public abstract class BeamPythonFunctionRunner implements PythonFunctionRunner { /** The Python process execution environment manager. */ private final ProcessPythonEnvironmentManager environmentManager; - /** The options used to configure the Python worker process. */ - private final Map jobOptions; - /** The flinkMetricContainer will be set to null if metric is configured to be turned off. */ - @Nullable private FlinkMetricContainer flinkMetricContainer; + @Nullable private final FlinkMetricContainer flinkMetricContainer; + + @Nullable private final KeyedStateBackend keyedStateBackend; + + @Nullable private final TypeSerializer keySerializer; + + @Nullable private final TypeSerializer namespaceSerializer; - @Nullable private TimerRegistration timerRegistration; + @Nullable private final TimerRegistration timerRegistration; private final MemoryManager memoryManager; @@ -145,7 +149,7 @@ public abstract class BeamPythonFunctionRunner implements PythonFunctionRunner { private transient StageBundleFactory stageBundleFactory; /** Handler for state requests. */ - private final StateRequestHandler stateRequestHandler; + private transient StateRequestHandler stateRequestHandler; /** Handler for bundle progress messages, both during bundle execution and on its completion. */ private transient BundleProgressHandler progressHandler; @@ -178,11 +182,10 @@ public abstract class BeamPythonFunctionRunner implements PythonFunctionRunner { public BeamPythonFunctionRunner( String taskName, ProcessPythonEnvironmentManager environmentManager, - Map jobOptions, @Nullable FlinkMetricContainer flinkMetricContainer, - @Nullable KeyedStateBackend keyedStateBackend, - @Nullable TypeSerializer keySerializer, - @Nullable TypeSerializer namespaceSerializer, + @Nullable KeyedStateBackend keyedStateBackend, + @Nullable TypeSerializer keySerializer, + @Nullable TypeSerializer namespaceSerializer, @Nullable TimerRegistration timerRegistration, MemoryManager memoryManager, double managedMemoryFraction, @@ -190,11 +193,10 @@ public BeamPythonFunctionRunner( FlinkFnApi.CoderInfoDescriptor outputCoderDescriptor) { this.taskName = Preconditions.checkNotNull(taskName); this.environmentManager = Preconditions.checkNotNull(environmentManager); - this.jobOptions = Preconditions.checkNotNull(jobOptions); this.flinkMetricContainer = flinkMetricContainer; - this.stateRequestHandler = - getStateRequestHandler( - keyedStateBackend, keySerializer, namespaceSerializer, jobOptions); + this.keyedStateBackend = keyedStateBackend; + this.keySerializer = keySerializer; + this.namespaceSerializer = namespaceSerializer; this.timerRegistration = timerRegistration; this.memoryManager = memoryManager; this.managedMemoryFraction = managedMemoryFraction; @@ -205,42 +207,45 @@ public BeamPythonFunctionRunner( // ------------------------------------------------------------------------ @Override - public void open(PythonConfig config) throws Exception { + public void open(ReadableConfig config) throws Exception { this.bundleStarted = false; this.resultBuffer = new LinkedBlockingQueue<>(); this.reusableResultTuple = new Tuple2<>(); + stateRequestHandler = + getStateRequestHandler( + keyedStateBackend, keySerializer, namespaceSerializer, config); + // The creation of stageBundleFactory depends on the initialized environment manager. environmentManager.open(); PortablePipelineOptions portableOptions = PipelineOptionsFactory.as(PortablePipelineOptions.class); - if (jobOptions.containsKey(PythonOptions.STATE_CACHE_SIZE.key())) { + int stateCacheSize = config.get(PythonOptions.STATE_CACHE_SIZE); + if (stateCacheSize > 0) { portableOptions .as(ExperimentalOptions.class) .setExperiments( Collections.singletonList( - ExperimentalOptions.STATE_CACHE_SIZE - + "=" - + jobOptions.get( - PythonOptions.STATE_CACHE_SIZE.key()))); + ExperimentalOptions.STATE_CACHE_SIZE + "=" + stateCacheSize)); } Struct pipelineOptions = PipelineOptionsTranslation.toProto(portableOptions); - if (memoryManager != null && config.isUsingManagedMemory()) { + if (memoryManager != null && config.get(USE_MANAGED_MEMORY)) { Preconditions.checkArgument( managedMemoryFraction > 0 && managedMemoryFraction <= 1.0, - "The configured managed memory fraction for Python worker process must be within (0, 1], was: %s. " - + "It may be because the consumer type \"Python\" was missing or set to 0 for the config option \"taskmanager.memory.managed.consumer-weights\"." - + managedMemoryFraction); + String.format( + "The configured managed memory fraction for Python worker process must be within (0, 1], was: %s. " + + "It may be because the consumer type \"Python\" was missing or set to 0 for the config option \"taskmanager.memory.managed.consumer-weights\".", + managedMemoryFraction)); final LongFunctionWithException initializer = (size) -> new PythonSharedResources( createJobBundleFactory(pipelineOptions), - createPythonExecutionEnvironment(size)); + createPythonExecutionEnvironment(config, size)); sharedResources = memoryManager.getSharedMemoryResourceForManagedMemory( @@ -259,7 +264,7 @@ public void open(PythonConfig config) throws Exception { jobBundleFactory = createJobBundleFactory(pipelineOptions); stageBundleFactory = createStageBundleFactory( - jobBundleFactory, createPythonExecutionEnvironment(-1)); + jobBundleFactory, createPythonExecutionEnvironment(config, -1)); } progressHandler = getProgressHandler(flinkMetricContainer); } @@ -390,13 +395,13 @@ private void finishBundle() { * Creates a specification which specifies the portability Python execution environment. It's * used by Beam's portability framework to creates the actual Python execution environment. */ - private RunnerApi.Environment createPythonExecutionEnvironment(long memoryLimitBytes) - throws Exception { + private RunnerApi.Environment createPythonExecutionEnvironment( + ReadableConfig config, long memoryLimitBytes) throws Exception { PythonEnvironment environment = environmentManager.createEnvironment(); if (environment instanceof ProcessPythonEnvironment) { ProcessPythonEnvironment processEnvironment = (ProcessPythonEnvironment) environment; Map env = processEnvironment.getEnv(); - env.putAll(jobOptions); + config.getOptional(PythonOptions.PYTHON_JOB_OPTIONS).ifPresent(env::putAll); env.put(PYTHON_WORKER_MEMORY_LIMIT, String.valueOf(memoryLimitBytes)); return Environments.createProcessEnvironment( "", "", processEnvironment.getCommand(), env); @@ -596,16 +601,16 @@ private TimerReceiverFactory createTimerReceiverFactory() { } private static StateRequestHandler getStateRequestHandler( - KeyedStateBackend keyedStateBackend, - TypeSerializer keySerializer, - TypeSerializer namespaceSerializer, - Map jobOptions) { + KeyedStateBackend keyedStateBackend, + TypeSerializer keySerializer, + TypeSerializer namespaceSerializer, + ReadableConfig config) { if (keyedStateBackend == null) { return StateRequestHandler.unsupported(); } else { assert keySerializer != null; return new SimpleStateRequestHandler( - keyedStateBackend, keySerializer, namespaceSerializer, jobOptions); + keyedStateBackend, keySerializer, namespaceSerializer, config); } } } diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/SimpleStateRequestHandler.java b/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/SimpleStateRequestHandler.java index 6f5e7524b721b..5eef9b7a427cb 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/SimpleStateRequestHandler.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/SimpleStateRequestHandler.java @@ -29,6 +29,7 @@ import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.typeutils.runtime.RowSerializer; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ByteArrayInputStreamWithPos; import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos; import org.apache.flink.core.memory.DataInputViewStreamWrapper; @@ -104,8 +105,8 @@ public class SimpleStateRequestHandler implements StateRequestHandler { BeamFnApi.StateGetResponse.newBuilder() .setData(ByteString.copyFrom(new byte[] {NOT_EMPTY_FLAG})); - private final TypeSerializer keySerializer; - private final TypeSerializer namespaceSerializer; + private final TypeSerializer keySerializer; + private final TypeSerializer namespaceSerializer; private final TypeSerializer valueSerializer; private final KeyedStateBackend keyedStateBackend; @@ -135,12 +136,12 @@ public class SimpleStateRequestHandler implements StateRequestHandler { private final BeamFnApi.ProcessBundleRequest.CacheToken cacheToken; SimpleStateRequestHandler( - KeyedStateBackend keyedStateBackend, - TypeSerializer keySerializer, - TypeSerializer namespaceSerializer, - Map config) { + KeyedStateBackend keyedStateBackend, + TypeSerializer keySerializer, + TypeSerializer namespaceSerializer, + ReadableConfig config) { this.keyedStateBackend = keyedStateBackend; - TypeSerializer frameworkKeySerializer = keyedStateBackend.getKeySerializer(); + TypeSerializer frameworkKeySerializer = keyedStateBackend.getKeySerializer(); if (!(frameworkKeySerializer instanceof AbstractRowDataSerializer || frameworkKeySerializer instanceof RowSerializer)) { throw new RuntimeException("Currently SimpleStateRequestHandler only support row key!"); @@ -157,12 +158,7 @@ public class SimpleStateRequestHandler implements StateRequestHandler { stateDescriptorCache = new HashMap<>(); mapStateIteratorCache = new HashMap<>(); mapStateIterateResponseBatchSize = - Integer.valueOf( - config.getOrDefault( - PythonOptions.MAP_STATE_ITERATE_RESPONSE_BATCH_SIZE.key(), - PythonOptions.MAP_STATE_ITERATE_RESPONSE_BATCH_SIZE - .defaultValue() - .toString())); + config.get(PythonOptions.MAP_STATE_ITERATE_RESPONSE_BATCH_SIZE); if (mapStateIterateResponseBatchSize <= 0) { throw new RuntimeException( String.format( diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/utils/ProtoUtils.java b/flink-python/src/main/java/org/apache/flink/streaming/api/utils/ProtoUtils.java index 6676d84235ba2..e5e482a6aa196 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/utils/ProtoUtils.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/utils/ProtoUtils.java @@ -132,7 +132,12 @@ public static FlinkFnApi.UserDefinedDataStreamFunction createUserDefinedDataStre DataStreamPythonFunctionInfo dataStreamPythonFunctionInfo, RuntimeContext runtimeContext, Map internalParameters, - boolean inBatchExecutionMode) { + boolean inBatchExecutionMode, + boolean isMetricEnabled, + boolean isProfileEnabled, + int stateCacheSize, + int mapStateReadCacheSize, + int mapStateWriteCacheSize) { FlinkFnApi.UserDefinedDataStreamFunction.Builder builder = FlinkFnApi.UserDefinedDataStreamFunction.newBuilder(); builder.setFunctionType( @@ -175,7 +180,11 @@ public static FlinkFnApi.UserDefinedDataStreamFunction createUserDefinedDataStre dataStreamPythonFunctionInfo .getPythonFunction() .getSerializedPythonFunction())); - builder.setMetricEnabled(true); + builder.setMetricEnabled(isMetricEnabled); + builder.setProfileEnabled(isProfileEnabled); + builder.setStateCacheSize(stateCacheSize); + builder.setMapStateReadCacheSize(mapStateReadCacheSize); + builder.setMapStateWriteCacheSize(mapStateWriteCacheSize); return builder.build(); } @@ -192,7 +201,12 @@ public static FlinkFnApi.UserDefinedDataStreamFunction createUserDefinedDataStre DataStreamPythonFunctionInfo dataStreamPythonFunctionInfo, RuntimeContext runtimeContext, Map internalParameters, - boolean inBatchExecutionMode) { + boolean inBatchExecutionMode, + boolean isMetricEnabled, + boolean isProfileEnabled, + int stateCacheSize, + int mapStateReadCacheSize, + int mapStateWriteCacheSize) { List results = new ArrayList<>(); Object[] inputs = dataStreamPythonFunctionInfo.getInputs(); @@ -203,7 +217,12 @@ public static FlinkFnApi.UserDefinedDataStreamFunction createUserDefinedDataStre (DataStreamPythonFunctionInfo) inputs[0], runtimeContext, internalParameters, - inBatchExecutionMode)); + inBatchExecutionMode, + isMetricEnabled, + isProfileEnabled, + stateCacheSize, + mapStateReadCacheSize, + mapStateWriteCacheSize)); } results.add( @@ -211,7 +230,12 @@ public static FlinkFnApi.UserDefinedDataStreamFunction createUserDefinedDataStre dataStreamPythonFunctionInfo, runtimeContext, internalParameters, - inBatchExecutionMode)); + inBatchExecutionMode, + isMetricEnabled, + isProfileEnabled, + stateCacheSize, + mapStateReadCacheSize, + mapStateWriteCacheSize)); return results; } @@ -221,13 +245,23 @@ public static FlinkFnApi.UserDefinedDataStreamFunction createUserDefinedDataStre RuntimeContext runtimeContext, Map internalParameters, TypeInformation keyTypeInfo, - boolean inBatchExecutionMode) { + boolean inBatchExecutionMode, + boolean isMetricEnabled, + boolean isProfileEnabled, + int stateCacheSize, + int mapStateReadCacheSize, + int mapStateWriteCacheSize) { List results = createUserDefinedDataStreamFunctionProtos( dataStreamPythonFunctionInfo, runtimeContext, internalParameters, - inBatchExecutionMode); + inBatchExecutionMode, + isMetricEnabled, + isProfileEnabled, + stateCacheSize, + mapStateReadCacheSize, + mapStateWriteCacheSize); // set the key typeinfo for the head operator FlinkFnApi.TypeInfo builtKeyTypeInfo = diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/utils/PythonOperatorUtils.java b/flink-python/src/main/java/org/apache/flink/streaming/api/utils/PythonOperatorUtils.java index f929ef9273bea..dc26f452ab5e5 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/utils/PythonOperatorUtils.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/utils/PythonOperatorUtils.java @@ -20,6 +20,8 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.runtime.state.KeyedStateBackend; +import org.apache.flink.streaming.api.operators.InternalTimerService; +import org.apache.flink.streaming.api.operators.sorted.state.BatchExecutionInternalTimeService; import org.apache.flink.streaming.api.operators.sorted.state.BatchExecutionKeyedStateBackend; /** Utilities used by Python operators. */ @@ -34,6 +36,15 @@ public static void setCurrentKeyForStreaming( } } + /** Set the current key for the timer service. */ + public static void setCurrentKeyForTimerService( + InternalTimerService internalTimerService, K currentKey) throws Exception { + if (internalTimerService instanceof BatchExecutionInternalTimeService) { + ((BatchExecutionInternalTimeService) internalTimerService) + .setCurrentKey(currentKey); + } + } + public static boolean inBatchExecutionMode(KeyedStateBackend stateBackend) { return stateBackend instanceof BatchExecutionKeyedStateBackend; } 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 46448bfc9e3a7..5e8b09fc4253c 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 @@ -567,7 +567,7 @@ private static boolean isStreamingMode(Table table) { TableEnvironment tableEnv = ((TableImpl) table).getTableEnvironment(); if (tableEnv instanceof TableEnvironmentImpl) { final RuntimeExecutionMode mode = - tableEnv.getConfig().getConfiguration().get(ExecutionOptions.RUNTIME_MODE); + tableEnv.getConfig().get(ExecutionOptions.RUNTIME_MODE); if (mode == RuntimeExecutionMode.AUTOMATIC) { throw new RuntimeException( String.format("Runtime execution mode '%s' is not supported yet.", mode)); diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/AbstractStatelessFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/AbstractStatelessFunctionOperator.java index b925bbe4426da..c5f9f66a42d6d 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/AbstractStatelessFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/AbstractStatelessFunctionOperator.java @@ -109,7 +109,6 @@ public PythonFunctionRunner createPythonFunctionRunner() throws IOException { createPythonEnvironmentManager(), getFunctionUrn(), getUserDefinedFunctionsProto(), - jobOptions, getFlinkMetricContainer(), getContainingTask().getEnvironment().getMemoryManager(), getOperatorConfig() 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 9ec99126be8c5..816ea04f3a227 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 @@ -47,6 +47,8 @@ import java.util.Arrays; import java.util.stream.Collectors; +import static org.apache.flink.python.PythonOptions.PYTHON_METRIC_ENABLED; +import static org.apache.flink.python.PythonOptions.PYTHON_PROFILE_ENABLED; import static org.apache.flink.streaming.api.utils.ProtoUtils.createRowTypeCoderInfoDescriptorProto; import static org.apache.flink.table.runtime.typeutils.PythonTypeUtils.toProtoType; @@ -156,7 +158,6 @@ public void open() throws Exception { PythonTypeUtils.toInternalSerializer(userDefinedFunctionOutputType); rowDataWrapper = new StreamRecordRowDataWrappingCollector(output); super.open(); - configJobOptions(); } @Override @@ -175,7 +176,6 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { createPythonEnvironmentManager(), getFunctionUrn(), getUserDefinedFunctionsProto(), - jobOptions, getFlinkMetricContainer(), getKeyedStateBackend(), getKeySerializer(), @@ -235,8 +235,8 @@ TypeSerializer getWindowSerializer() { protected FlinkFnApi.UserDefinedAggregateFunctions getUserDefinedFunctionsProto() { FlinkFnApi.UserDefinedAggregateFunctions.Builder builder = FlinkFnApi.UserDefinedAggregateFunctions.newBuilder(); - builder.setMetricEnabled(pythonConfig.isMetricEnabled()); - builder.setProfileEnabled(pythonConfig.isProfileEnabled()); + builder.setMetricEnabled(config.get(PYTHON_METRIC_ENABLED)); + builder.setProfileEnabled(config.get(PYTHON_PROFILE_ENABLED)); builder.addAllGrouping(Arrays.stream(grouping).boxed().collect(Collectors.toList())); builder.setGenerateUpdateBefore(generateUpdateBefore); builder.setIndexOfCountStar(indexOfCountStar); @@ -263,15 +263,6 @@ protected FlinkFnApi.UserDefinedAggregateFunctions getUserDefinedFunctionsProto( public abstract RowType createUserDefinedFunctionOutputType(); - private void configJobOptions() { - jobOptions.put( - PythonOptions.STATE_CACHE_SIZE.key(), - String.valueOf(config.get(PythonOptions.STATE_CACHE_SIZE))); - jobOptions.put( - PythonOptions.MAP_STATE_ITERATE_RESPONSE_BATCH_SIZE.key(), - String.valueOf(config.get(PythonOptions.MAP_STATE_ITERATE_RESPONSE_BATCH_SIZE))); - } - public FlinkFnApi.CoderInfoDescriptor createInputCoderInfoDescriptor(RowType runnerInputType) { return createRowTypeCoderInfoDescriptorProto( runnerInputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, false); diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/AbstractArrowPythonAggregateFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/AbstractArrowPythonAggregateFunctionOperator.java index a893bb8617f8b..cd7167f1da9f1 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/AbstractArrowPythonAggregateFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/AbstractArrowPythonAggregateFunctionOperator.java @@ -36,6 +36,8 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Preconditions; +import static org.apache.flink.python.PythonOptions.PYTHON_METRIC_ENABLED; +import static org.apache.flink.python.PythonOptions.PYTHON_PROFILE_ENABLED; import static org.apache.flink.streaming.api.utils.ProtoUtils.createArrowTypeCoderInfoDescriptorProto; import static org.apache.flink.streaming.api.utils.ProtoUtils.getUserDefinedFunctionProto; @@ -153,8 +155,8 @@ public FlinkFnApi.UserDefinedFunctions getUserDefinedFunctionsProto() { for (PythonFunctionInfo pythonFunctionInfo : pandasAggFunctions) { builder.addUdfs(getUserDefinedFunctionProto(pythonFunctionInfo)); } - builder.setMetricEnabled(pythonConfig.isMetricEnabled()); - builder.setProfileEnabled(pythonConfig.isProfileEnabled()); + builder.setMetricEnabled(config.get(PYTHON_METRIC_ENABLED)); + builder.setProfileEnabled(config.get(PYTHON_PROFILE_ENABLED)); return builder.build(); } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonOverWindowAggregateFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonOverWindowAggregateFunctionOperator.java index 2fd586131e3fa..1425a35fe9548 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonOverWindowAggregateFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonOverWindowAggregateFunctionOperator.java @@ -37,6 +37,8 @@ import java.util.List; import java.util.ListIterator; +import static org.apache.flink.python.PythonOptions.PYTHON_METRIC_ENABLED; +import static org.apache.flink.python.PythonOptions.PYTHON_PROFILE_ENABLED; import static org.apache.flink.streaming.api.utils.ProtoUtils.createOverWindowArrowTypeCoderInfoDescriptorProto; import static org.apache.flink.streaming.api.utils.ProtoUtils.getUserDefinedFunctionProto; @@ -259,8 +261,8 @@ public FlinkFnApi.UserDefinedFunctions getUserDefinedFunctionsProto() { functionBuilder.setWindowIndex(aggWindowIndex[i]); builder.addUdfs(functionBuilder); } - builder.setMetricEnabled(pythonConfig.isMetricEnabled()); - builder.setProfileEnabled(pythonConfig.isProfileEnabled()); + builder.setMetricEnabled(config.get(PYTHON_METRIC_ENABLED)); + builder.setProfileEnabled(config.get(PYTHON_PROFILE_ENABLED)); // add windows for (int i = 0; i < lowerBoundary.length; i++) { FlinkFnApi.OverWindow.Builder windowBuilder = FlinkFnApi.OverWindow.newBuilder(); diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/AbstractPythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/AbstractPythonScalarFunctionOperator.java index e5220fb005529..ed887657e2147 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/AbstractPythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/AbstractPythonScalarFunctionOperator.java @@ -35,6 +35,9 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Preconditions; +import static org.apache.flink.python.PythonOptions.PYTHON_METRIC_ENABLED; +import static org.apache.flink.python.PythonOptions.PYTHON_PROFILE_ENABLED; + /** * Base class for all stream operators to execute Python {@link ScalarFunction}s. It executes the * Python {@link ScalarFunction}s in separate Python execution environment. @@ -121,8 +124,8 @@ public FlinkFnApi.UserDefinedFunctions getUserDefinedFunctionsProto() { for (PythonFunctionInfo pythonFunctionInfo : scalarFunctions) { builder.addUdfs(ProtoUtils.getUserDefinedFunctionProto(pythonFunctionInfo)); } - builder.setMetricEnabled(pythonConfig.isMetricEnabled()); - builder.setProfileEnabled(pythonConfig.isProfileEnabled()); + builder.setMetricEnabled(config.get(PYTHON_METRIC_ENABLED)); + builder.setProfileEnabled(config.get(PYTHON_PROFILE_ENABLED)); return builder.build(); } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/EmbeddedPythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/EmbeddedPythonScalarFunctionOperator.java index 848b6785057a8..e30180bcb98e4 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/EmbeddedPythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/EmbeddedPythonScalarFunctionOperator.java @@ -40,17 +40,13 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Preconditions; -import pemja.core.PythonInterpreterConfig; - -import java.io.BufferedInputStream; -import java.io.BufferedReader; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; import java.util.Arrays; import java.util.Map; import java.util.stream.Collectors; +import static org.apache.flink.python.PythonOptions.PYTHON_METRIC_ENABLED; +import static org.apache.flink.python.PythonOptions.PYTHON_PROFILE_ENABLED; + /** The Python {@link ScalarFunction} operator in embedded Python environment. */ @Internal public class EmbeddedPythonScalarFunctionOperator @@ -156,46 +152,21 @@ public void open() throws Exception { } @Override - public void openPythonInterpreter( - String pythonExecutable, - Map env, - PythonInterpreterConfig.ExecType execType) - throws Exception { - if (execType.equals(PythonInterpreterConfig.ExecType.SUB_INTERPRETER)) { - LOG.info("Create Operation in sub-interpreters."); - String[] commands = - new String[] { - pythonExecutable, - "-c", - String.format( - "from pyflink.fn_execution.utils.operation_utils import create_serialized_scalar_operation_from_proto;" - + "print(create_serialized_scalar_operation_from_proto(%s, %s, %s))", - Arrays.toString(getUserDefinedFunctionsProto().toByteArray()), - isOneArg ? "True" : "False", - isOneFieldResult ? "True" : "False") - }; - interpreter.exec( - "from pyflink.fn_execution.utils.operation_utils import deserialized_operation_from_serialized_bytes"); - interpreter.exec( - String.format( - "scalar_operation = deserialized_operation_from_serialized_bytes(%s)", - executeScript(commands, env))); - } else { - LOG.info("Create Operation in multi-threads."); - - // The CPython extension included in proto does not support initialization - // multiple times, so we choose the only interpreter process to be responsible for - // initialization and proto parsing. The only interpreter parses the proto and - // serializes function operations with cloudpickle. - interpreter.exec( - "from pyflink.fn_execution.utils.operation_utils import create_scalar_operation_from_proto"); - interpreter.set("proto", getUserDefinedFunctionsProto().toByteArray()); - - interpreter.exec( - String.format( - "scalar_operation = create_scalar_operation_from_proto(proto, %s, %s)", - isOneArg ? "True" : "False", isOneFieldResult ? "True" : "False")); - } + public void openPythonInterpreter(String pythonExecutable, Map env) { + LOG.info("Create Operation in multi-threads."); + + // The CPython extension included in proto does not support initialization + // multiple times, so we choose the only interpreter process to be responsible for + // initialization and proto parsing. The only interpreter parses the proto and + // serializes function operations with cloudpickle. + interpreter.exec( + "from pyflink.fn_execution.utils.operation_utils import create_scalar_operation_from_proto"); + interpreter.set("proto", getUserDefinedFunctionsProto().toByteArray()); + + interpreter.exec( + String.format( + "scalar_operation = create_scalar_operation_from_proto(proto, %s, %s)", + isOneArg ? "True" : "False", isOneFieldResult ? "True" : "False")); // invoke `open` method of ScalarOperation. interpreter.invokeMethod("scalar_operation", "open"); @@ -268,36 +239,8 @@ public FlinkFnApi.UserDefinedFunctions getUserDefinedFunctionsProto() { for (PythonFunctionInfo pythonFunctionInfo : scalarFunctions) { builder.addUdfs(ProtoUtils.getUserDefinedFunctionProto(pythonFunctionInfo)); } - builder.setMetricEnabled(pythonConfig.isMetricEnabled()); - builder.setProfileEnabled(pythonConfig.isProfileEnabled()); + builder.setMetricEnabled(config.get(PYTHON_METRIC_ENABLED)); + builder.setProfileEnabled(config.get(PYTHON_PROFILE_ENABLED)); return builder.build(); } - - private String executeScript(final String[] commands, Map env) - throws IOException { - ProcessBuilder pb = new ProcessBuilder(commands); - pb.environment().putAll(env); - pb.redirectErrorStream(true); - Process p = pb.start(); - InputStream in = new BufferedInputStream(p.getInputStream()); - StringBuilder out = new StringBuilder(); - String s; - try (BufferedReader br = new BufferedReader(new InputStreamReader(in))) { - while ((s = br.readLine()) != null) { - out.append(s).append("\n"); - } - } - try { - if (p.waitFor() != 0) { - throw new IOException( - String.format( - "Failed to execute the command: %s\noutput: %s", - String.join(" ", commands), out)); - } - } catch (InterruptedException e) { - // Ignored. The subprocess is dead after "br.readLine()" returns null, so the call of - // "waitFor" should return intermediately. - } - return out.toString(); - } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperator.java index 1db9dda484a69..37e9010f22753 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperator.java @@ -30,6 +30,7 @@ import org.apache.flink.table.runtime.operators.python.scalar.AbstractPythonScalarFunctionOperator; import org.apache.flink.table.types.logical.RowType; +import static org.apache.flink.python.PythonOptions.MAX_ARROW_BATCH_SIZE; import static org.apache.flink.streaming.api.utils.ProtoUtils.createArrowTypeCoderInfoDescriptorProto; /** Arrow Python {@link ScalarFunction} operator. */ @@ -67,7 +68,7 @@ public ArrowPythonScalarFunctionOperator( @Override public void open() throws Exception { super.open(); - maxArrowBatchSize = Math.min(pythonConfig.getMaxArrowBatchSize(), maxBundleSize); + maxArrowBatchSize = Math.min(config.get(MAX_ARROW_BATCH_SIZE), maxBundleSize); arrowSerializer = new ArrowSerializer(udfInputType, udfOutputType); arrowSerializer.open(bais, baos); currentBatchCount = 0; diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperator.java index 43778cfeb5b9b..81862b993007b 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperator.java @@ -41,6 +41,8 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Preconditions; +import static org.apache.flink.python.PythonOptions.PYTHON_METRIC_ENABLED; +import static org.apache.flink.python.PythonOptions.PYTHON_PROFILE_ENABLED; import static org.apache.flink.streaming.api.utils.ProtoUtils.createFlattenRowTypeCoderInfoDescriptorProto; import static org.apache.flink.streaming.api.utils.ProtoUtils.createRowTypeCoderInfoDescriptorProto; @@ -157,8 +159,8 @@ public FlinkFnApi.UserDefinedFunctions getUserDefinedFunctionsProto() { FlinkFnApi.UserDefinedFunctions.Builder builder = FlinkFnApi.UserDefinedFunctions.newBuilder(); builder.addUdfs(ProtoUtils.getUserDefinedFunctionProto(tableFunction)); - builder.setMetricEnabled(pythonConfig.isMetricEnabled()); - builder.setProfileEnabled(pythonConfig.isProfileEnabled()); + builder.setMetricEnabled(config.get(PYTHON_METRIC_ENABLED)); + builder.setProfileEnabled(config.get(PYTHON_PROFILE_ENABLED)); return builder.build(); } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/beam/BeamTablePythonFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/beam/BeamTablePythonFunctionRunner.java index d502a47c01651..ff605356d3465 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/beam/BeamTablePythonFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/beam/BeamTablePythonFunctionRunner.java @@ -34,7 +34,6 @@ import java.util.Collections; import java.util.List; -import java.util.Map; import java.util.Optional; import static org.apache.flink.python.Constants.INPUT_COLLECTION_ID; @@ -57,7 +56,6 @@ public BeamTablePythonFunctionRunner( ProcessPythonEnvironmentManager environmentManager, String functionUrn, GeneratedMessageV3 userDefinedFunctionProto, - Map jobOptions, FlinkMetricContainer flinkMetricContainer, KeyedStateBackend keyedStateBackend, TypeSerializer keySerializer, @@ -69,7 +67,6 @@ public BeamTablePythonFunctionRunner( super( taskName, environmentManager, - jobOptions, flinkMetricContainer, keyedStateBackend, keySerializer, @@ -122,7 +119,6 @@ public static BeamTablePythonFunctionRunner stateless( ProcessPythonEnvironmentManager environmentManager, String functionUrn, GeneratedMessageV3 userDefinedFunctionProto, - Map jobOptions, FlinkMetricContainer flinkMetricContainer, MemoryManager memoryManager, double managedMemoryFraction, @@ -133,7 +129,6 @@ public static BeamTablePythonFunctionRunner stateless( environmentManager, functionUrn, userDefinedFunctionProto, - jobOptions, flinkMetricContainer, null, null, @@ -149,7 +144,6 @@ public static BeamTablePythonFunctionRunner stateful( ProcessPythonEnvironmentManager environmentManager, String functionUrn, GeneratedMessageV3 userDefinedFunctionProto, - Map jobOptions, FlinkMetricContainer flinkMetricContainer, KeyedStateBackend keyedStateBackend, TypeSerializer keySerializer, @@ -163,7 +157,6 @@ public static BeamTablePythonFunctionRunner stateful( environmentManager, functionUrn, userDefinedFunctionProto, - jobOptions, flinkMetricContainer, keyedStateBackend, keySerializer, 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 8d7a83246fc1e..29abc5e561e18 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 @@ -64,7 +64,6 @@ import java.math.BigDecimal; import java.math.RoundingMode; import java.sql.Time; -import java.util.TimeZone; /** * Utilities for converting Flink logical types, such as convert it to the related TypeSerializer or @@ -75,12 +74,6 @@ public final class PythonTypeUtils { private static final String EMPTY_STRING = ""; - /** The local time zone. */ - private static final TimeZone LOCAL_TZ = TimeZone.getDefault(); - - /** The number of milliseconds in a day. */ - private static final long MILLIS_PER_DAY = 86400000L; // = 24 * 60 * 60 * 1000 - public static FlinkFnApi.Schema.FieldType toProtoType(LogicalType logicalType) { return logicalType.accept(new PythonTypeUtils.LogicalTypeToProtoTypeConverter()); } diff --git a/flink-python/src/main/resources/META-INF/NOTICE b/flink-python/src/main/resources/META-INF/NOTICE index 8532b8f5dcfc1..182971f352073 100644 --- a/flink-python/src/main/resources/META-INF/NOTICE +++ b/flink-python/src/main/resources/META-INF/NOTICE @@ -6,9 +6,9 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) -- com.fasterxml.jackson.core:jackson-annotations:2.13.0 -- com.fasterxml.jackson.core:jackson-core:2.13.0 -- com.fasterxml.jackson.core:jackson-databind:2.13.0 +- com.fasterxml.jackson.core:jackson-annotations:2.13.2 +- com.fasterxml.jackson.core:jackson-core:2.13.2 +- com.fasterxml.jackson.core:jackson-databind:2.13.2.2 - com.google.flatbuffers:flatbuffers-java:1.9.0 - io.netty:netty-buffer:4.1.70.Final - io.netty:netty-common:4.1.70.Final @@ -27,7 +27,7 @@ This project bundles the following dependencies under the Apache Software Licens - org.apache.beam:beam-vendor-sdks-java-extensions-protobuf:2.27.0 - org.apache.beam:beam-vendor-guava-26_0-jre:0.1 - org.apache.beam:beam-vendor-grpc-1_26_0:0.3 -- com.alibaba:pemja:0.1.2 +- com.alibaba:pemja:0.1.4 This project bundles the following dependencies under the BSD license. See bundled license files for details diff --git a/flink-python/src/test/java/org/apache/flink/client/python/PythonDriverTest.java b/flink-python/src/test/java/org/apache/flink/client/python/PythonDriverTest.java index 773353d71afe0..fe46868618224 100644 --- a/flink-python/src/test/java/org/apache/flink/client/python/PythonDriverTest.java +++ b/flink-python/src/test/java/org/apache/flink/client/python/PythonDriverTest.java @@ -65,11 +65,12 @@ public void testConstructCommandsWithEntryPointScript() { args.add("--input"); args.add("in.txt"); - PythonDriverOptions pythonDriverOptions = new PythonDriverOptions(null, "xxx", args); + PythonDriverOptions pythonDriverOptions = new PythonDriverOptions(null, "xxx.py", args); List commands = PythonDriver.constructPythonCommands(pythonDriverOptions); - Assert.assertEquals(3, commands.size()); - Assert.assertEquals(commands.get(0), "xxx"); - Assert.assertEquals(commands.get(1), "--input"); - Assert.assertEquals(commands.get(2), "in.txt"); + Assert.assertEquals(4, commands.size()); + Assert.assertEquals(commands.get(0), "-m"); + Assert.assertEquals(commands.get(1), "xxx"); + Assert.assertEquals(commands.get(2), "--input"); + Assert.assertEquals(commands.get(3), "in.txt"); } } diff --git a/flink-python/src/test/java/org/apache/flink/client/python/PythonFunctionFactoryTest.java b/flink-python/src/test/java/org/apache/flink/client/python/PythonFunctionFactoryTest.java index b51cf07f171fb..87da19bdd2a75 100644 --- a/flink-python/src/test/java/org/apache/flink/client/python/PythonFunctionFactoryTest.java +++ b/flink-python/src/test/java/org/apache/flink/client/python/PythonFunctionFactoryTest.java @@ -17,6 +17,7 @@ package org.apache.flink.client.python; +import org.apache.flink.configuration.MemorySize; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.TableEnvironment; @@ -69,8 +70,8 @@ public static void prepareEnvironment() throws Exception { } StreamExecutionEnvironment sEnv = StreamExecutionEnvironment.getExecutionEnvironment(); tableEnv = StreamTableEnvironment.create(sEnv); - tableEnv.getConfig().getConfiguration().set(PYTHON_FILES, pyFilePath.getAbsolutePath()); - tableEnv.getConfig().getConfiguration().setString(TASK_OFF_HEAP_MEMORY.key(), "80mb"); + tableEnv.getConfig().set(PYTHON_FILES, pyFilePath.getAbsolutePath()); + tableEnv.getConfig().set(TASK_OFF_HEAP_MEMORY, MemorySize.parse("80mb")); sourceTable = tableEnv.fromDataStream(sEnv.fromElements("1", "2", "3")).as("str"); } diff --git a/flink-python/src/test/java/org/apache/flink/python/PythonConfigTest.java b/flink-python/src/test/java/org/apache/flink/python/PythonConfigTest.java deleted file mode 100644 index 871c862282a63..0000000000000 --- a/flink-python/src/test/java/org/apache/flink/python/PythonConfigTest.java +++ /dev/null @@ -1,146 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.python; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.python.util.PythonDependencyUtils; - -import org.junit.Test; - -import java.util.HashMap; -import java.util.Map; - -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.is; -import static org.junit.Assert.assertThat; - -/** Tests for {@link PythonConfig}. */ -public class PythonConfigTest { - - @Test - public void testDefaultConfigure() { - PythonConfig pythonConfig = new PythonConfig(new Configuration()); - assertThat( - pythonConfig.getMaxBundleSize(), - is(equalTo(PythonOptions.MAX_BUNDLE_SIZE.defaultValue()))); - assertThat( - pythonConfig.getMaxBundleTimeMills(), - is(equalTo(PythonOptions.MAX_BUNDLE_TIME_MILLS.defaultValue()))); - assertThat( - pythonConfig.getMaxArrowBatchSize(), - is(equalTo(PythonOptions.MAX_ARROW_BATCH_SIZE.defaultValue()))); - assertThat(pythonConfig.getPythonFilesInfo().isEmpty(), is(true)); - assertThat(pythonConfig.getPythonRequirementsFileInfo().isPresent(), is(false)); - assertThat(pythonConfig.getPythonRequirementsCacheDirInfo().isPresent(), is(false)); - assertThat(pythonConfig.getPythonArchivesInfo().isEmpty(), is(true)); - assertThat(pythonConfig.getPythonExec(), is("python")); - assertThat( - pythonConfig.isUsingManagedMemory(), - is(equalTo(PythonOptions.USE_MANAGED_MEMORY.defaultValue()))); - } - - @Test - public void testMaxBundleSize() { - Configuration config = new Configuration(); - config.set(PythonOptions.MAX_BUNDLE_SIZE, 10); - PythonConfig pythonConfig = new PythonConfig(config); - assertThat(pythonConfig.getMaxBundleSize(), is(equalTo(10))); - } - - @Test - public void testMaxBundleTimeMills() { - Configuration config = new Configuration(); - config.set(PythonOptions.MAX_BUNDLE_TIME_MILLS, 10L); - PythonConfig pythonConfig = new PythonConfig(config); - assertThat(pythonConfig.getMaxBundleTimeMills(), is(equalTo(10L))); - } - - @Test - public void testMaxArrowBatchSize() { - Configuration config = new Configuration(); - config.set(PythonOptions.MAX_ARROW_BATCH_SIZE, 10); - PythonConfig pythonConfig = new PythonConfig(config); - assertThat(pythonConfig.getMaxArrowBatchSize(), is(equalTo(10))); - } - - @Test - public void testPythonFilesInfo() { - Configuration config = new Configuration(); - Map pythonFiles = new HashMap<>(); - pythonFiles.put("python_file_{SHA256}", "file0.py"); - config.set(PythonDependencyUtils.PYTHON_FILES, pythonFiles); - PythonConfig pythonConfig = new PythonConfig(config); - assertThat(pythonConfig.getPythonFilesInfo(), is(equalTo(pythonFiles))); - } - - @Test - public void testPythonRequirementsFileInfo() { - Configuration config = new Configuration(); - Map pythonRequirementsFile = - config.getOptional(PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE) - .orElse(new HashMap<>()); - pythonRequirementsFile.put(PythonDependencyUtils.FILE, "python_requirements_file_{SHA256}"); - config.set(PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE, pythonRequirementsFile); - PythonConfig pythonConfig = new PythonConfig(config); - assertThat( - pythonConfig.getPythonRequirementsFileInfo().get(), - is(equalTo("python_requirements_file_{SHA256}"))); - } - - @Test - public void testPythonRequirementsCacheDirInfo() { - Configuration config = new Configuration(); - Map pythonRequirementsFile = - config.getOptional(PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE) - .orElse(new HashMap<>()); - pythonRequirementsFile.put( - PythonDependencyUtils.CACHE, "python_requirements_cache_{SHA256}"); - config.set(PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE, pythonRequirementsFile); - PythonConfig pythonConfig = new PythonConfig(config); - assertThat( - pythonConfig.getPythonRequirementsCacheDirInfo().get(), - is(equalTo("python_requirements_cache_{SHA256}"))); - } - - @Test - public void testPythonArchivesInfo() { - Configuration config = new Configuration(); - Map pythonArchives = new HashMap<>(); - pythonArchives.put("python_archive_{SHA256}", "file0.zip"); - config.set(PythonDependencyUtils.PYTHON_ARCHIVES, pythonArchives); - PythonConfig pythonConfig = new PythonConfig(config); - assertThat(pythonConfig.getPythonArchivesInfo(), is(equalTo(pythonArchives))); - } - - @Test - public void testPythonExec() { - Configuration config = new Configuration(); - config.set(PythonOptions.PYTHON_EXECUTABLE, "/usr/local/bin/python3"); - PythonConfig pythonConfig = new PythonConfig(config); - assertThat(pythonConfig.getPythonExec(), is(equalTo("/usr/local/bin/python3"))); - } - - @Test - public void testManagedMemory() { - Configuration config = new Configuration(); - config.set(PythonOptions.USE_MANAGED_MEMORY, true); - PythonConfig pythonConfig = new PythonConfig(config); - assertThat(pythonConfig.isUsingManagedMemory(), is(equalTo(true))); - } -} diff --git a/flink-python/src/test/java/org/apache/flink/python/env/PythonDependencyInfoTest.java b/flink-python/src/test/java/org/apache/flink/python/env/PythonDependencyInfoTest.java index b80f56f0e07de..5478130b1b01f 100644 --- a/flink-python/src/test/java/org/apache/flink/python/env/PythonDependencyInfoTest.java +++ b/flink-python/src/test/java/org/apache/flink/python/env/PythonDependencyInfoTest.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; -import org.apache.flink.python.PythonConfig; import org.apache.flink.python.PythonOptions; import org.apache.flink.python.util.PythonDependencyUtils; import org.apache.flink.util.OperatingSystem; @@ -35,6 +34,9 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; +import static org.apache.flink.python.PythonOptions.PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO; +import static org.apache.flink.python.PythonOptions.PYTHON_FILES_DISTRIBUTED_CACHE_INFO; +import static org.apache.flink.python.PythonOptions.PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -75,9 +77,8 @@ public void testParsePythonFiles() { Map pythonFiles = new HashMap<>(); pythonFiles.put("python_file_{SHA256_0}", "test_file1.py"); pythonFiles.put("python_file_{SHA256_1}", "test_file2.py"); - config.set(PythonDependencyUtils.PYTHON_FILES, pythonFiles); - PythonDependencyInfo dependencyInfo = - PythonDependencyInfo.create(new PythonConfig(config), distributedCache); + config.set(PYTHON_FILES_DISTRIBUTED_CACHE_INFO, pythonFiles); + PythonDependencyInfo dependencyInfo = PythonDependencyInfo.create(config, distributedCache); Map expected = new HashMap<>(); expected.put("/distributed_cache/file0", "test_file1.py"); @@ -91,18 +92,17 @@ public void testParsePythonRequirements() throws IOException { Assume.assumeFalse(OperatingSystem.isWindows()); Configuration config = new Configuration(); - config.set(PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE, new HashMap<>()); - config.get(PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE) + config.set(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO, new HashMap<>()); + config.get(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO) .put(PythonDependencyUtils.FILE, "python_requirements_file_{SHA256}"); - PythonDependencyInfo dependencyInfo = - PythonDependencyInfo.create(new PythonConfig(config), distributedCache); + PythonDependencyInfo dependencyInfo = PythonDependencyInfo.create(config, distributedCache); assertEquals("/distributed_cache/file2", dependencyInfo.getRequirementsFilePath().get()); assertFalse(dependencyInfo.getRequirementsCacheDir().isPresent()); - config.get(PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE) + config.get(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO) .put(PythonDependencyUtils.CACHE, "python_requirements_cache_{SHA256}"); - dependencyInfo = PythonDependencyInfo.create(new PythonConfig(config), distributedCache); + dependencyInfo = PythonDependencyInfo.create(config, distributedCache); assertEquals("/distributed_cache/file2", dependencyInfo.getRequirementsFilePath().get()); assertEquals("/distributed_cache/file3", dependencyInfo.getRequirementsCacheDir().get()); @@ -117,9 +117,8 @@ public void testParsePythonArchives() { Map pythonArchives = new HashMap<>(); pythonArchives.put("python_archive_{SHA256_0}", "py27.zip"); pythonArchives.put("python_archive_{SHA256_1}", "py37"); - config.set(PythonDependencyUtils.PYTHON_ARCHIVES, pythonArchives); - PythonDependencyInfo dependencyInfo = - PythonDependencyInfo.create(new PythonConfig(config), distributedCache); + config.set(PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO, pythonArchives); + PythonDependencyInfo dependencyInfo = PythonDependencyInfo.create(config, distributedCache); Map expected = new HashMap<>(); expected.put("/distributed_cache/file4", "py27.zip"); @@ -131,8 +130,7 @@ public void testParsePythonArchives() { public void testParsePythonExec() { Configuration config = new Configuration(); config.set(PythonOptions.PYTHON_EXECUTABLE, "/usr/bin/python3"); - PythonDependencyInfo dependencyInfo = - PythonDependencyInfo.create(new PythonConfig(config), distributedCache); + PythonDependencyInfo dependencyInfo = PythonDependencyInfo.create(config, distributedCache); assertEquals("/usr/bin/python3", dependencyInfo.getPythonExec()); } diff --git a/flink-python/src/test/java/org/apache/flink/python/util/PythonConfigUtilTest.java b/flink-python/src/test/java/org/apache/flink/python/util/PythonConfigUtilTest.java index c6e997c620788..2718634b479e4 100644 --- a/flink-python/src/test/java/org/apache/flink/python/util/PythonConfigUtilTest.java +++ b/flink-python/src/test/java/org/apache/flink/python/util/PythonConfigUtilTest.java @@ -25,25 +25,13 @@ import org.junit.Test; -import java.lang.reflect.InvocationTargetException; import java.util.Collections; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; /** A test class to test PythonConfigUtil getting executionEnvironment correctly. */ public class PythonConfigUtilTest { - @Test - public void testGetEnvironmentConfig() - throws IllegalAccessException, NoSuchFieldException, InvocationTargetException { - StreamExecutionEnvironment executionEnvironment = - StreamExecutionEnvironment.getExecutionEnvironment(); - Configuration envConfig = - PythonConfigUtil.getEnvConfigWithDependencies(executionEnvironment); - assertNotNull(envConfig); - } - @Test public void testJobName() { String jobName = "MyTestJob"; diff --git a/flink-python/src/test/java/org/apache/flink/python/util/PythonDependencyUtilsTest.java b/flink-python/src/test/java/org/apache/flink/python/util/PythonDependencyUtilsTest.java index 46f370be76802..01238c08a8626 100644 --- a/flink-python/src/test/java/org/apache/flink/python/util/PythonDependencyUtilsTest.java +++ b/flink-python/src/test/java/org/apache/flink/python/util/PythonDependencyUtilsTest.java @@ -32,14 +32,14 @@ import java.util.Properties; import java.util.stream.Collectors; +import static org.apache.flink.python.PythonOptions.PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO; import static org.apache.flink.python.PythonOptions.PYTHON_CLIENT_EXECUTABLE; import static org.apache.flink.python.PythonOptions.PYTHON_EXECUTABLE; +import static org.apache.flink.python.PythonOptions.PYTHON_FILES_DISTRIBUTED_CACHE_INFO; import static org.apache.flink.python.PythonOptions.PYTHON_REQUIREMENTS; +import static org.apache.flink.python.PythonOptions.PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO; import static org.apache.flink.python.util.PythonDependencyUtils.CACHE; import static org.apache.flink.python.util.PythonDependencyUtils.FILE; -import static org.apache.flink.python.util.PythonDependencyUtils.PYTHON_ARCHIVES; -import static org.apache.flink.python.util.PythonDependencyUtils.PYTHON_FILES; -import static org.apache.flink.python.util.PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE; import static org.apache.flink.python.util.PythonDependencyUtils.configurePythonDependencies; import static org.apache.flink.python.util.PythonDependencyUtils.merge; import static org.junit.Assert.assertEquals; @@ -74,20 +74,20 @@ public void testPythonFiles() { "tmp_dir/test_dir"); verifyCachedFiles(expectedCachedFiles); - Configuration expectedConfiguration = new Configuration(config); - expectedConfiguration.set(PYTHON_FILES, new HashMap<>()); + Configuration expectedConfiguration = new Configuration(); + expectedConfiguration.set(PYTHON_FILES_DISTRIBUTED_CACHE_INFO, new HashMap<>()); expectedConfiguration - .get(PYTHON_FILES) + .get(PYTHON_FILES_DISTRIBUTED_CACHE_INFO) .put( "python_file_83bbdaee494ad7d9b334c02ec71dc86a0868f7f8e49d1249a37c517dc6ee15a7", "test_file1.py"); expectedConfiguration - .get(PYTHON_FILES) + .get(PYTHON_FILES_DISTRIBUTED_CACHE_INFO) .put( "python_file_e57a895cb1256500098be0874128680cd9f56000d48fcd393c48d6371bd2d947", "test_file2.py"); expectedConfiguration - .get(PYTHON_FILES) + .get(PYTHON_FILES_DISTRIBUTED_CACHE_INFO) .put( "python_file_e56bc55ff643576457b3d012b2bba888727c71cf05a958930f2263398c4e9798", "test_dir"); @@ -106,10 +106,10 @@ public void testPythonRequirements() { "tmp_dir/requirements.txt"); verifyCachedFiles(expectedCachedFiles); - Configuration expectedConfiguration = new Configuration(config); - expectedConfiguration.set(PYTHON_REQUIREMENTS_FILE, new HashMap<>()); + Configuration expectedConfiguration = new Configuration(); + expectedConfiguration.set(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO, new HashMap<>()); expectedConfiguration - .get(PYTHON_REQUIREMENTS_FILE) + .get(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO) .put( FILE, "python_requirements_file_69390ca43c69ada3819226fcfbb5b6d27e111132a9427e7f201edd82e9d65ff6"); @@ -127,15 +127,15 @@ public void testPythonRequirements() { "tmp_dir/cache"); verifyCachedFiles(expectedCachedFiles); - expectedConfiguration = new Configuration(config); - expectedConfiguration.set(PYTHON_REQUIREMENTS_FILE, new HashMap<>()); + expectedConfiguration = new Configuration(); + expectedConfiguration.set(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO, new HashMap<>()); expectedConfiguration - .get(PYTHON_REQUIREMENTS_FILE) + .get(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO) .put( FILE, "python_requirements_file_56fd0c530faaa7129dca8d314cf69cbfc7c1c5c952f5176a003253e2f418873e"); expectedConfiguration - .get(PYTHON_REQUIREMENTS_FILE) + .get(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO) .put( CACHE, "python_requirements_cache_2f563dd6731c2c7c5e1ef1ef8279f61e907dc3bfc698adb71b109e43ed93e143"); @@ -169,25 +169,25 @@ public void testPythonArchives() { "tmp_dir/py37.zip"); verifyCachedFiles(expectedCachedFiles); - Configuration expectedConfiguration = new Configuration(config); - expectedConfiguration.set(PYTHON_ARCHIVES, new HashMap<>()); + Configuration expectedConfiguration = new Configuration(); + expectedConfiguration.set(PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO, new HashMap<>()); expectedConfiguration - .get(PYTHON_ARCHIVES) + .get(PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO) .put( "python_archive_4cc74e4003de886434723f351771df2a84f72531c52085acc0915e19d70df2ba", "file1.zip"); expectedConfiguration - .get(PYTHON_ARCHIVES) + .get(PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO) .put( "python_archive_5f3fca2a4165c7d9c94b00bfab956c15f14c41e9e03f6037c83eb61157fce09c", "py37.zip"); expectedConfiguration - .get(PYTHON_ARCHIVES) + .get(PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO) .put( "python_archive_f8a1c874251230f21094880d9dd878ffb5714454b69184d8ad268a6563269f0c", "py37.zip#venv2"); expectedConfiguration - .get(PYTHON_ARCHIVES) + .get(PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO) .put( "python_archive_c7d970ce1c5794367974ce8ef536c2343bed8fcfe7c2422c51548e58007eee6a", "py37.zip#venv"); diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PassThroughPythonStreamGroupWindowAggregateOperator.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PassThroughPythonStreamGroupWindowAggregateOperator.java index a5bf90a43d148..8f328680e81a8 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PassThroughPythonStreamGroupWindowAggregateOperator.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PassThroughPythonStreamGroupWindowAggregateOperator.java @@ -201,7 +201,6 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { userDefinedFunctionOutputType, STREAM_GROUP_WINDOW_AGGREGATE_URN, getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer(), getKeyedStateBackend(), getKeySerializer(), diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupAggregateOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupAggregateOperatorTest.java index d5472fc27a505..2e6e9f377d84f 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupAggregateOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupAggregateOperatorTest.java @@ -40,7 +40,6 @@ import org.junit.Test; import java.io.IOException; -import java.util.HashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.function.Function; @@ -262,7 +261,6 @@ public PythonFunctionRunner createPythonFunctionRunner() { outputType, STREAM_GROUP_AGGREGATE_URN, getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer(), getKeyedStateBackend(), getKeySerializer(), diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupTableAggregateOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupTableAggregateOperatorTest.java index bdef22d144c1e..d4cffdb493598 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupTableAggregateOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupTableAggregateOperatorTest.java @@ -40,7 +40,6 @@ import org.junit.Test; import java.io.IOException; -import java.util.HashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.function.Function; @@ -275,7 +274,6 @@ public PythonFunctionRunner createPythonFunctionRunner() { outputType, STREAM_GROUP_TABLE_AGGREGATE_URN, getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer(), getKeyedStateBackend(), getKeySerializer(), diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupAggregateFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupAggregateFunctionOperatorTest.java index e7581199b21cf..52981006ba122 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupAggregateFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupAggregateFunctionOperatorTest.java @@ -42,7 +42,6 @@ import org.junit.Test; import java.util.Arrays; -import java.util.HashMap; import java.util.concurrent.ConcurrentLinkedQueue; /** @@ -238,7 +237,6 @@ public PythonFunctionRunner createPythonFunctionRunner() { udfOutputType, getFunctionUrn(), getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer(), false); } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupWindowAggregateFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupWindowAggregateFunctionOperatorTest.java index a6ec6db0c7802..0812090f1d7c2 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupWindowAggregateFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupWindowAggregateFunctionOperatorTest.java @@ -44,7 +44,6 @@ import org.junit.Test; import java.util.Arrays; -import java.util.HashMap; import java.util.concurrent.ConcurrentLinkedQueue; /** @@ -407,7 +406,6 @@ public PythonFunctionRunner createPythonFunctionRunner() { udfOutputType, getFunctionUrn(), getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer(), false); } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonOverWindowAggregateFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonOverWindowAggregateFunctionOperatorTest.java index 1fee4769471da..276bae1f6a415 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonOverWindowAggregateFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonOverWindowAggregateFunctionOperatorTest.java @@ -43,7 +43,6 @@ import org.junit.Test; import java.util.Arrays; -import java.util.HashMap; import java.util.List; import java.util.concurrent.ConcurrentLinkedQueue; @@ -308,7 +307,6 @@ public PythonFunctionRunner createPythonFunctionRunner() { udfOutputType, getFunctionUrn(), getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer(), true); } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonGroupWindowAggregateFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonGroupWindowAggregateFunctionOperatorTest.java index e7622221adc0b..be4bdabdb4830 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonGroupWindowAggregateFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonGroupWindowAggregateFunctionOperatorTest.java @@ -55,7 +55,6 @@ import java.time.Duration; import java.time.ZoneId; import java.util.Arrays; -import java.util.HashMap; import java.util.concurrent.ConcurrentLinkedQueue; /** @@ -522,7 +521,6 @@ public PythonFunctionRunner createPythonFunctionRunner() { udfOutputType, getFunctionUrn(), getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer(), false); } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonProcTimeBoundedRangeOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonProcTimeBoundedRangeOperatorTest.java index a95157c1277c6..5d85019a7a2e2 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonProcTimeBoundedRangeOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonProcTimeBoundedRangeOperatorTest.java @@ -41,7 +41,6 @@ import org.junit.Test; import java.util.Arrays; -import java.util.HashMap; import java.util.concurrent.ConcurrentLinkedQueue; /** Test for {@link StreamArrowPythonProcTimeBoundedRangeOperator}. */ @@ -168,7 +167,6 @@ public PythonFunctionRunner createPythonFunctionRunner() { udfOutputType, getFunctionUrn(), getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer(), false); } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonProcTimeBoundedRowsOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonProcTimeBoundedRowsOperatorTest.java index b65a0fb226f99..117c40d1b4e8f 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonProcTimeBoundedRowsOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonProcTimeBoundedRowsOperatorTest.java @@ -41,7 +41,6 @@ import org.junit.Test; import java.util.Arrays; -import java.util.HashMap; import java.util.concurrent.ConcurrentLinkedQueue; /** Test for {@link StreamArrowPythonProcTimeBoundedRowsOperator}. */ @@ -169,7 +168,6 @@ public PythonFunctionRunner createPythonFunctionRunner() { udfOutputType, getFunctionUrn(), getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer(), false); } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonRowTimeBoundedRangeOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonRowTimeBoundedRangeOperatorTest.java index 849c6b0a8a15b..620de10884432 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonRowTimeBoundedRangeOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonRowTimeBoundedRangeOperatorTest.java @@ -45,7 +45,6 @@ import org.junit.Test; import java.util.Arrays; -import java.util.HashMap; import java.util.concurrent.ConcurrentLinkedQueue; import static org.junit.Assert.assertEquals; @@ -324,7 +323,6 @@ public PythonFunctionRunner createPythonFunctionRunner() { udfOutputType, getFunctionUrn(), getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer(), false); } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonRowTimeBoundedRowsOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonRowTimeBoundedRowsOperatorTest.java index 1ee0e7f1d81cc..dafdf3b54349b 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonRowTimeBoundedRowsOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonRowTimeBoundedRowsOperatorTest.java @@ -43,7 +43,6 @@ import org.junit.Test; import java.util.Arrays; -import java.util.HashMap; import java.util.concurrent.ConcurrentLinkedQueue; /** @@ -294,7 +293,6 @@ public PythonFunctionRunner createPythonFunctionRunner() { udfOutputType, getFunctionUrn(), getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer(), false); } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTest.java index 17ae33cbbd518..25f53d94166b1 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTest.java @@ -41,7 +41,6 @@ import java.io.IOException; import java.util.Collection; -import java.util.HashMap; import static org.apache.flink.table.runtime.util.StreamRecordUtils.row; @@ -152,7 +151,6 @@ public PythonFunctionRunner createPythonFunctionRunner() throws IOException { udfOutputType, getFunctionUrn(), getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer()); } } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTestBase.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTestBase.java index 8abfbc60707f6..cc752d3e4d397 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTestBase.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTestBase.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.MemorySize; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.python.PythonOptions; @@ -218,9 +219,7 @@ public void testPythonScalarFunctionOperatorIsChainedByDefault() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(1); StreamTableEnvironment tEnv = createTableEnvironment(env); - tEnv.getConfig() - .getConfiguration() - .setString(TaskManagerOptions.TASK_OFF_HEAP_MEMORY.key(), "80mb"); + tEnv.getConfig().set(TaskManagerOptions.TASK_OFF_HEAP_MEMORY, MemorySize.parse("80mb")); tEnv.registerFunction("pyFunc", new PythonScalarFunction("pyFunc")); DataStream> ds = env.fromElements(new Tuple2<>(1, 2)); Table t = tEnv.fromDataStream(ds, $("a"), $("b")).select(call("pyFunc", $("a"), $("b"))); diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperatorTest.java index 953afb0e3be33..e42a9c12dab6f 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperatorTest.java @@ -42,7 +42,6 @@ import java.io.IOException; import java.util.Collection; -import java.util.HashMap; import static org.apache.flink.table.runtime.util.StreamRecordUtils.row; @@ -151,7 +150,6 @@ public PythonFunctionRunner createPythonFunctionRunner() throws IOException { udfOutputType, getFunctionUrn(), getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer()); } } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperatorTest.java index 840503c2737cc..8b2b0b0d1620e 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperatorTest.java @@ -40,7 +40,6 @@ import org.apache.calcite.rel.core.JoinRelType; import java.util.Collection; -import java.util.HashMap; import static org.apache.flink.table.runtime.util.StreamRecordUtils.row; @@ -133,7 +132,6 @@ public PythonFunctionRunner createPythonFunctionRunner() { udfOutputType, getFunctionUrn(), getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer()); } } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonAggregateFunctionRunner.java b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonAggregateFunctionRunner.java index fb16068280957..f19b5c60188f5 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonAggregateFunctionRunner.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonAggregateFunctionRunner.java @@ -19,11 +19,11 @@ package org.apache.flink.table.runtime.utils; import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ByteArrayInputStreamWithPos; import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos; import org.apache.flink.core.memory.DataInputViewStreamWrapper; import org.apache.flink.fnexecution.v1.FlinkFnApi; -import org.apache.flink.python.PythonConfig; import org.apache.flink.python.env.process.ProcessPythonEnvironmentManager; import org.apache.flink.python.metric.FlinkMetricContainer; import org.apache.flink.table.data.RowData; @@ -37,7 +37,6 @@ import java.util.ArrayList; import java.util.LinkedList; import java.util.List; -import java.util.Map; import static org.apache.flink.streaming.api.utils.ProtoUtils.createArrowTypeCoderInfoDescriptorProto; @@ -72,7 +71,6 @@ public PassThroughPythonAggregateFunctionRunner( RowType outputType, String functionUrn, FlinkFnApi.UserDefinedFunctions userDefinedFunctions, - Map jobOptions, FlinkMetricContainer flinkMetricContainer, boolean isBatchOverWindow) { super( @@ -80,7 +78,6 @@ public PassThroughPythonAggregateFunctionRunner( environmentManager, functionUrn, userDefinedFunctions, - jobOptions, flinkMetricContainer, null, null, @@ -97,7 +94,7 @@ public PassThroughPythonAggregateFunctionRunner( } @Override - public void open(PythonConfig config) throws Exception { + public void open(ReadableConfig config) throws Exception { super.open(config); bais = new ByteArrayInputStreamWithPos(); baisWrapper = new DataInputViewStreamWrapper(bais); diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonScalarFunctionRunner.java b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonScalarFunctionRunner.java index 4815507f6aa65..e0cd0fbde1c07 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonScalarFunctionRunner.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonScalarFunctionRunner.java @@ -29,7 +29,6 @@ import java.util.LinkedList; import java.util.List; -import java.util.Map; import static org.apache.flink.streaming.api.utils.ProtoUtils.createFlattenRowTypeCoderInfoDescriptorProto; @@ -48,14 +47,12 @@ public PassThroughPythonScalarFunctionRunner( RowType outputType, String functionUrn, FlinkFnApi.UserDefinedFunctions userDefinedFunctions, - Map jobOptions, FlinkMetricContainer flinkMetricContainer) { super( taskName, environmentManager, functionUrn, userDefinedFunctions, - jobOptions, flinkMetricContainer, null, null, diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonTableFunctionRunner.java b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonTableFunctionRunner.java index 94694222b498b..48af1207b6e35 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonTableFunctionRunner.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonTableFunctionRunner.java @@ -29,7 +29,6 @@ import java.util.LinkedList; import java.util.List; -import java.util.Map; import static org.apache.flink.streaming.api.utils.ProtoUtils.createFlattenRowTypeCoderInfoDescriptorProto; @@ -50,14 +49,12 @@ public PassThroughPythonTableFunctionRunner( RowType outputType, String functionUrn, FlinkFnApi.UserDefinedFunctions userDefinedFunctions, - Map jobOptions, FlinkMetricContainer flinkMetricContainer) { super( taskName, environmentManager, functionUrn, userDefinedFunctions, - jobOptions, flinkMetricContainer, null, null, diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamAggregatePythonFunctionRunner.java b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamAggregatePythonFunctionRunner.java index e2886005c5400..230f2e4b86261 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamAggregatePythonFunctionRunner.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamAggregatePythonFunctionRunner.java @@ -31,7 +31,6 @@ import java.util.LinkedList; import java.util.List; -import java.util.Map; import java.util.function.Function; import static org.apache.flink.streaming.api.utils.ProtoUtils.createRowTypeCoderInfoDescriptorProto; @@ -54,7 +53,6 @@ public PassThroughStreamAggregatePythonFunctionRunner( RowType outputType, String functionUrn, FlinkFnApi.UserDefinedAggregateFunctions userDefinedFunctions, - Map jobOptions, FlinkMetricContainer flinkMetricContainer, KeyedStateBackend keyedStateBackend, TypeSerializer keySerializer, @@ -64,7 +62,6 @@ public PassThroughStreamAggregatePythonFunctionRunner( environmentManager, functionUrn, userDefinedFunctions, - jobOptions, flinkMetricContainer, keyedStateBackend, keySerializer, diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamGroupWindowAggregatePythonFunctionRunner.java b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamGroupWindowAggregatePythonFunctionRunner.java index 7f977f2dbcf17..a76fc1c7fd845 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamGroupWindowAggregatePythonFunctionRunner.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamGroupWindowAggregatePythonFunctionRunner.java @@ -30,8 +30,6 @@ import org.apache.beam.runners.fnexecution.control.JobBundleFactory; import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct; -import java.util.Map; - import static org.apache.flink.streaming.api.utils.ProtoUtils.createFlattenRowTypeCoderInfoDescriptorProto; /** @@ -50,7 +48,6 @@ public PassThroughStreamGroupWindowAggregatePythonFunctionRunner( RowType outputType, String functionUrn, FlinkFnApi.UserDefinedAggregateFunctions userDefinedFunctions, - Map jobOptions, FlinkMetricContainer flinkMetricContainer, KeyedStateBackend keyedStateBackend, TypeSerializer keySerializer, @@ -60,7 +57,6 @@ public PassThroughStreamGroupWindowAggregatePythonFunctionRunner( environmentManager, functionUrn, userDefinedFunctions, - jobOptions, flinkMetricContainer, keyedStateBackend, keySerializer, diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamTableAggregatePythonFunctionRunner.java b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamTableAggregatePythonFunctionRunner.java index f9ed8acefab86..d519a1a220f1d 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamTableAggregatePythonFunctionRunner.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamTableAggregatePythonFunctionRunner.java @@ -32,7 +32,6 @@ import java.util.Arrays; import java.util.LinkedList; import java.util.List; -import java.util.Map; import java.util.function.Function; import static org.apache.flink.streaming.api.utils.ProtoUtils.createRowTypeCoderInfoDescriptorProto; @@ -56,7 +55,6 @@ public PassThroughStreamTableAggregatePythonFunctionRunner( RowType outputType, String functionUrn, FlinkFnApi.UserDefinedAggregateFunctions userDefinedFunctions, - Map jobOptions, FlinkMetricContainer flinkMetricContainer, KeyedStateBackend keyedStateBackend, TypeSerializer keySerializer, @@ -66,7 +64,6 @@ public PassThroughStreamTableAggregatePythonFunctionRunner( environmentManager, functionUrn, userDefinedFunctions, - jobOptions, flinkMetricContainer, keyedStateBackend, keySerializer, diff --git a/flink-python/tox.ini b/flink-python/tox.ini index 7a6a934b0f78c..44149024bbe08 100644 --- a/flink-python/tox.ini +++ b/flink-python/tox.ini @@ -48,7 +48,7 @@ install_command = {toxinidir}/dev/install_command.sh {opts} {packages} # up to 100 characters in length, not 79. ignore=E226,E241,E305,E402,E722,E731,E741,W503,W504 max-line-length=100 -exclude=.tox/*,dev/*,lib/*,target/*,build/*,dist/*,pyflink/shell.py,.eggs/*,pyflink/fn_execution/tests/process_mode_test_data.py,pyflink/fn_execution/*_pb2.py +exclude=.tox/*,dev/*,lib/*,target/*,build/*,dist/*,pyflink/shell.py,.eggs/*,pyflink/fn_execution/tests/process_mode_test_data.py,pyflink/fn_execution/*_pb2.py,pyflink/examples/table/basic_operations.py [mypy] files=pyflink/common/*.py,pyflink/table/*.py,pyflink/datastream/*.py,pyflink/metrics/*.py diff --git a/flink-queryable-state/flink-queryable-state-client-java/pom.xml b/flink-queryable-state/flink-queryable-state-client-java/pom.xml index 8566c24172311..be384d304e933 100644 --- a/flink-queryable-state/flink-queryable-state-client-java/pom.xml +++ b/flink-queryable-state/flink-queryable-state-client-java/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-queryable-state - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-queryable-state/flink-queryable-state-runtime/pom.xml b/flink-queryable-state/flink-queryable-state-runtime/pom.xml index 75de79c11c6bb..8cc2eb9cbb3d4 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/pom.xml +++ b/flink-queryable-state/flink-queryable-state-runtime/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-queryable-state - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-queryable-state/pom.xml b/flink-queryable-state/pom.xml index f0f85230da932..9ba1e593dceee 100644 --- a/flink-queryable-state/pom.xml +++ b/flink-queryable-state/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-quickstart/flink-quickstart-java/pom.xml b/flink-quickstart/flink-quickstart-java/pom.xml index c532434215a47..0dbc7dd8012fd 100644 --- a/flink-quickstart/flink-quickstart-java/pom.xml +++ b/flink-quickstart/flink-quickstart-java/pom.xml @@ -27,7 +27,7 @@ under the License. org.apache.flink flink-quickstart - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-quickstart/flink-quickstart-scala/pom.xml b/flink-quickstart/flink-quickstart-scala/pom.xml index a0feb5e51c7fa..573a729e35537 100644 --- a/flink-quickstart/flink-quickstart-scala/pom.xml +++ b/flink-quickstart/flink-quickstart-scala/pom.xml @@ -27,7 +27,7 @@ under the License. org.apache.flink flink-quickstart - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-quickstart/pom.xml b/flink-quickstart/pom.xml index c6010915ab182..ef0fac1251293 100644 --- a/flink-quickstart/pom.xml +++ b/flink-quickstart/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-rpc/flink-rpc-akka-loader/pom.xml b/flink-rpc/flink-rpc-akka-loader/pom.xml index 1b0dedaf1af07..e4fd77f82b2ba 100644 --- a/flink-rpc/flink-rpc-akka-loader/pom.xml +++ b/flink-rpc/flink-rpc-akka-loader/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-rpc - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-rpc/flink-rpc-akka/pom.xml b/flink-rpc/flink-rpc-akka/pom.xml index 5d028ae5c58ef..36fc16bd3de47 100644 --- a/flink-rpc/flink-rpc-akka/pom.xml +++ b/flink-rpc/flink-rpc-akka/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-rpc - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-rpc/flink-rpc-core/pom.xml b/flink-rpc/flink-rpc-core/pom.xml index c781c7e162f48..08c8ab9b3eaba 100644 --- a/flink-rpc/flink-rpc-core/pom.xml +++ b/flink-rpc/flink-rpc-core/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-rpc - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-rpc/pom.xml b/flink-rpc/pom.xml index 0eed91f9f2c1f..d9ee427f6767f 100644 --- a/flink-rpc/pom.xml +++ b/flink-rpc/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-runtime-web/pom.xml b/flink-runtime-web/pom.xml index f6eabd5bf3b51..4420d5fa2cdb0 100644 --- a/flink-runtime-web/pom.xml +++ b/flink-runtime-web/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml index b728d8d92d0eb..9f84e6fe65d68 100644 --- a/flink-runtime/pom.xml +++ b/flink-runtime/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index 2efc034ca90f8..72a6b7032deba 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -1780,7 +1780,8 @@ public boolean restoreSavepoint( // register shared state - even before adding the checkpoint to the store // because the latter might trigger subsumption so the ref counts must be up-to-date savepoint.registerSharedStatesAfterRestored( - completedCheckpointStore.getSharedStateRegistry()); + completedCheckpointStore.getSharedStateRegistry(), + restoreSettings.getRestoreMode()); completedCheckpointStore.addCheckpointAndSubsumeOldestOne( savepoint, checkpointsCleaner, this::scheduleTriggerRequest); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointFailureManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointFailureManager.java index 5b73fe29d8aad..08cf49e41ee34 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointFailureManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointFailureManager.java @@ -104,12 +104,20 @@ public void handleCheckpointException( : pendingCheckpoint.getCheckpointID(); updateStatsAfterCheckpointFailed(pendingCheckpointStats, statsTracker, exception); - LOG.warn( - "Failed to trigger or complete checkpoint {} for job {}. ({} consecutive failed attempts so far)", - checkpointId == UNKNOWN_CHECKPOINT_ID ? "UNKNOWN_CHECKPOINT_ID" : checkpointId, - job, - continuousFailureCounter.get(), - exception); + if (CheckpointFailureReason.NOT_ALL_REQUIRED_TASKS_RUNNING.equals( + exception.getCheckpointFailureReason())) { + LOG.info( + "Failed to trigger checkpoint for job {} since {}.", + job, + exception.getMessage()); + } else { + LOG.warn( + "Failed to trigger or complete checkpoint {} for job {}. ({} consecutive failed attempts so far)", + checkpointId == UNKNOWN_CHECKPOINT_ID ? "UNKNOWN_CHECKPOINT_ID" : checkpointId, + job, + continuousFailureCounter.get(), + exception); + } if (isJobManagerFailure(exception, executionAttemptID)) { handleJobLevelCheckpointException(checkpointProperties, exception, checkpointId); } else { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounter.java index 81e0471c4061b..bb4a39522b62c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounter.java @@ -20,6 +20,8 @@ import org.apache.flink.api.common.JobStatus; +import java.util.concurrent.CompletableFuture; + /** A checkpoint ID counter. */ public interface CheckpointIDCounter { int INITIAL_CHECKPOINT_ID = 1; @@ -34,8 +36,9 @@ public interface CheckpointIDCounter { * or kept. * * @param jobStatus Job state on shut down + * @return The {@code CompletableFuture} holding the result of the shutdown operation. */ - void shutdown(JobStatus jobStatus) throws Exception; + CompletableFuture shutdown(JobStatus jobStatus); /** * Atomically increments the current checkpoint ID. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRecoveryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRecoveryFactory.java index ab0b5ef8506ed..64c68caa8a9d0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRecoveryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRecoveryFactory.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.SharedStateRegistryFactory; @@ -37,13 +38,15 @@ public interface CheckpointRecoveryFactory { * @param sharedStateRegistryFactory Simple factory to produce {@link SharedStateRegistry} * objects. * @param ioExecutor Executor used to run (async) deletes. + * @param restoreMode the restore mode with which the job is restoring. * @return {@link CompletedCheckpointStore} instance for the job */ CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, SharedStateRegistryFactory sharedStateRegistryFactory, - Executor ioExecutor) + Executor ioExecutor, + RestoreMode restoreMode) throws Exception; /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java index 41c577452bb61..f0270ab661a7c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.StateUtil; @@ -207,11 +208,13 @@ public long getStateSize() { * checkpoint is added into the store. * * @param sharedStateRegistry The registry where shared states are registered + * @param restoreMode the mode in which this checkpoint was restored from */ - public void registerSharedStatesAfterRestored(SharedStateRegistry sharedStateRegistry) { + public void registerSharedStatesAfterRestored( + SharedStateRegistry sharedStateRegistry, RestoreMode restoreMode) { // in claim mode we should not register any shared handles if (!props.isUnclaimed()) { - sharedStateRegistry.registerAll(operatorStates.values(), checkpointID); + sharedStateRegistry.registerAllAfterRestored(this, restoreMode); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DeactivatedCheckpointIDCounter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DeactivatedCheckpointIDCounter.java index ca86159b6f581..7d670102063f2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DeactivatedCheckpointIDCounter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DeactivatedCheckpointIDCounter.java @@ -19,6 +19,9 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobStatus; +import org.apache.flink.util.concurrent.FutureUtils; + +import java.util.concurrent.CompletableFuture; /** * This class represents a {@link CheckpointIDCounter} if checkpointing is deactivated. @@ -32,7 +35,9 @@ public enum DeactivatedCheckpointIDCounter implements CheckpointIDCounter { public void start() throws Exception {} @Override - public void shutdown(JobStatus jobStatus) throws Exception {} + public CompletableFuture shutdown(JobStatus jobStatus) { + return FutureUtils.completedVoidFuture(); + } @Override public long getAndIncrement() throws Exception { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/EmbeddedCompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/EmbeddedCompletedCheckpointStore.java index 1e5e47c8855df..744083ca6afd9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/EmbeddedCompletedCheckpointStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/EmbeddedCompletedCheckpointStore.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobStatus; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.util.Preconditions; import org.apache.flink.util.concurrent.Executors; @@ -53,16 +54,22 @@ public EmbeddedCompletedCheckpointStore() { @VisibleForTesting public EmbeddedCompletedCheckpointStore(int maxRetainedCheckpoints) { - this(maxRetainedCheckpoints, Collections.emptyList()); + this( + maxRetainedCheckpoints, + Collections.emptyList(), + /* Using the default restore mode in tests to detect any breaking changes early. */ + RestoreMode.DEFAULT); } public EmbeddedCompletedCheckpointStore( - int maxRetainedCheckpoints, Collection initialCheckpoints) { + int maxRetainedCheckpoints, + Collection initialCheckpoints, + RestoreMode restoreMode) { this( maxRetainedCheckpoints, initialCheckpoints, SharedStateRegistry.DEFAULT_FACTORY.create( - Executors.directExecutor(), initialCheckpoints)); + Executors.directExecutor(), initialCheckpoints, restoreMode)); } public EmbeddedCompletedCheckpointStore( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryFactory.java index bc18c453969d1..7a70f5624a5b5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryFactory.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.state.SharedStateRegistryFactory; import javax.annotation.Nullable; @@ -42,7 +43,7 @@ public class PerJobCheckpointRecoveryFactory public static CheckpointRecoveryFactory withoutCheckpointStoreRecovery(IntFunction storeFn) { return new PerJobCheckpointRecoveryFactory<>( - (maxCheckpoints, previous, sharedStateRegistry, ioExecutor) -> { + (maxCheckpoints, previous, sharedStateRegistry, ioExecutor, restoreMode) -> { if (previous != null) { throw new UnsupportedOperationException( "Checkpoint store recovery is not supported."); @@ -75,7 +76,8 @@ public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, SharedStateRegistryFactory sharedStateRegistryFactory, - Executor ioExecutor) { + Executor ioExecutor, + RestoreMode restoreMode) { return store.compute( jobId, (key, previous) -> @@ -83,7 +85,8 @@ public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( maxNumberOfCheckpointsToRetain, previous, sharedStateRegistryFactory, - ioExecutor)); + ioExecutor, + restoreMode)); } @Override @@ -98,6 +101,7 @@ StoreType recoverCheckpointStore( int maxNumberOfCheckpointsToRetain, @Nullable StoreType previousStore, SharedStateRegistryFactory sharedStateRegistryFactory, - Executor ioExecutor); + Executor ioExecutor, + RestoreMode restoreMode); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointIDCounter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointIDCounter.java index 30b653949892b..82400350422eb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointIDCounter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointIDCounter.java @@ -20,7 +20,9 @@ import org.apache.flink.api.common.JobStatus; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; +import org.apache.flink.util.concurrent.FutureUtils; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicLong; /** @@ -37,7 +39,9 @@ public class StandaloneCheckpointIDCounter implements CheckpointIDCounter { public void start() throws Exception {} @Override - public void shutdown(JobStatus jobStatus) throws Exception {} + public CompletableFuture shutdown(JobStatus jobStatus) { + return FutureUtils.completedVoidFuture(); + } @Override public long getAndIncrement() throws Exception { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java index 95f9da72406ff..abcb704ad7c0a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import org.apache.flink.runtime.state.SharedStateRegistryFactory; @@ -32,11 +33,15 @@ public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, SharedStateRegistryFactory sharedStateRegistryFactory, - Executor ioExecutor) + Executor ioExecutor, + RestoreMode restoreMode) throws Exception { return new StandaloneCompletedCheckpointStore( - maxNumberOfCheckpointsToRetain, sharedStateRegistryFactory, ioExecutor); + maxNumberOfCheckpointsToRetain, + sharedStateRegistryFactory, + ioExecutor, + restoreMode); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java index 87a6486a911f6..6c89dcc712788 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobStatus; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.SharedStateRegistryFactory; @@ -56,32 +57,38 @@ public StandaloneCompletedCheckpointStore(int maxNumberOfCheckpointsToRetain) { this( maxNumberOfCheckpointsToRetain, SharedStateRegistry.DEFAULT_FACTORY, - Executors.directExecutor()); + Executors.directExecutor(), + /* Using the default restore mode in tests to detect any breaking changes early. */ + RestoreMode.DEFAULT); } /** * Creates {@link StandaloneCompletedCheckpointStore}. * + * @param restoreMode * @param maxNumberOfCheckpointsToRetain The maximum number of checkpoints to retain (at least * 1). Adding more checkpoints than this results in older checkpoints being discarded. */ public StandaloneCompletedCheckpointStore( int maxNumberOfCheckpointsToRetain, SharedStateRegistryFactory sharedStateRegistryFactory, - Executor ioExecutor) { + Executor ioExecutor, + RestoreMode restoreMode) { this( maxNumberOfCheckpointsToRetain, sharedStateRegistryFactory, new ArrayDeque<>(maxNumberOfCheckpointsToRetain + 1), - ioExecutor); + ioExecutor, + restoreMode); } private StandaloneCompletedCheckpointStore( int maxNumberOfCheckpointsToRetain, SharedStateRegistryFactory sharedStateRegistryFactory, ArrayDeque checkpoints, - Executor ioExecutor) { - super(sharedStateRegistryFactory.create(ioExecutor, checkpoints)); + Executor ioExecutor, + RestoreMode restoreMode) { + super(sharedStateRegistryFactory.create(ioExecutor, checkpoints, restoreMode)); checkArgument(maxNumberOfCheckpointsToRetain >= 1, "Must retain at least one checkpoint."); this.maxNumberOfCheckpointsToRetain = maxNumberOfCheckpointsToRetain; this.checkpoints = checkpoints; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounter.java index 9c649e6ac7627..6576dde11c21f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounter.java @@ -22,18 +22,27 @@ import org.apache.flink.api.common.JobStatus; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import org.apache.flink.runtime.util.ZooKeeperUtils; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.concurrent.FutureUtils; +import org.apache.flink.shaded.curator5.com.google.common.collect.Sets; import org.apache.flink.shaded.curator5.org.apache.curator.framework.CuratorFramework; +import org.apache.flink.shaded.curator5.org.apache.curator.framework.api.CuratorEvent; +import org.apache.flink.shaded.curator5.org.apache.curator.framework.api.CuratorEventType; import org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.shared.SharedCount; import org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.shared.VersionedValue; import org.apache.flink.shaded.curator5.org.apache.curator.framework.state.ConnectionState; +import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.concurrent.GuardedBy; +import java.io.IOException; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -101,22 +110,65 @@ public void start() throws Exception { } @Override - public void shutdown(JobStatus jobStatus) throws Exception { + public CompletableFuture shutdown(JobStatus jobStatus) { synchronized (startStopLock) { if (isStarted) { LOG.info("Shutting down."); - sharedCount.close(); + try { + sharedCount.close(); + } catch (IOException e) { + return FutureUtils.completedExceptionally(e); + } client.getConnectionStateListenable().removeListener(connectionStateListener); if (jobStatus.isGloballyTerminalState()) { LOG.info("Removing {} from ZooKeeper", counterPath); - client.delete().deletingChildrenIfNeeded().inBackground().forPath(counterPath); + try { + final CompletableFuture deletionFuture = new CompletableFuture<>(); + client.delete() + .inBackground( + (curatorFramework, curatorEvent) -> + handleDeletionOfCounterPath( + curatorEvent, deletionFuture)) + .forPath(counterPath); + return deletionFuture; + } catch (Exception e) { + return FutureUtils.completedExceptionally(e); + } } isStarted = false; } } + + return FutureUtils.completedVoidFuture(); + } + + private void handleDeletionOfCounterPath( + CuratorEvent curatorEvent, CompletableFuture deletionFuture) { + Preconditions.checkArgument( + curatorEvent.getType() == CuratorEventType.DELETE, + "An unexpected CuratorEvent was monitored: " + curatorEvent.getType()); + Preconditions.checkArgument( + counterPath.endsWith(curatorEvent.getPath()), + "An unexpected path was selected for deletion: " + curatorEvent.getPath()); + + final KeeperException.Code eventCode = + KeeperException.Code.get(curatorEvent.getResultCode()); + if (Sets.immutableEnumSet(KeeperException.Code.OK, KeeperException.Code.NONODE) + .contains(eventCode)) { + deletionFuture.complete(null); + } else { + final String namespacedCounterPath = + ZooKeeperUtils.generateZookeeperPath(client.getNamespace(), counterPath); + deletionFuture.completeExceptionally( + new FlinkException( + String.format( + "An error occurred while shutting down the CheckpointIDCounter in path '%s'.", + namespacedCounterPath), + KeeperException.create(eventCode, namespacedCounterPath))); + } } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java index 052b3405235ae..c522296cf89e9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import org.apache.flink.runtime.state.SharedStateRegistryFactory; import org.apache.flink.runtime.util.ZooKeeperUtils; @@ -51,7 +52,8 @@ public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, SharedStateRegistryFactory sharedStateRegistryFactory, - Executor ioExecutor) + Executor ioExecutor, + RestoreMode restoreMode) throws Exception { return ZooKeeperUtils.createCompletedCheckpoints( @@ -61,7 +63,8 @@ public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( maxNumberOfCheckpointsToRetain, sharedStateRegistryFactory, ioExecutor, - executor); + executor, + restoreMode); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index 393a4291544c5..e6c9e74b443bd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -24,8 +24,11 @@ import org.apache.flink.api.common.operators.ResourceSpec; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.BlobServerOptions; +import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.ClusterOptions; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.core.execution.SavepointFormatType; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.blob.BlobServer; @@ -44,6 +47,7 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.JobResultEntry; import org.apache.flink.runtime.highavailability.JobResultStore; +import org.apache.flink.runtime.highavailability.JobResultStoreOptions; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.OperatorID; @@ -608,14 +612,20 @@ private void runJob(JobManagerRunner jobManagerRunner, ExecutionType executionTy return handleJobManagerRunnerResult( jobManagerRunnerResult, executionType); } else { - return jobManagerRunnerFailed(jobId, throwable); + return CompletableFuture.completedFuture( + jobManagerRunnerFailed(jobId, throwable)); } }, - getMainThreadExecutor()); + getMainThreadExecutor()) + .thenCompose(Function.identity()); final CompletableFuture jobTerminationFuture = cleanupJobStateFuture.thenCompose( - cleanupJobState -> removeJob(jobId, cleanupJobState)); + cleanupJobState -> + removeJob(jobId, cleanupJobState) + .exceptionally( + throwable -> + logCleanupErrorWarning(jobId, throwable))); FutureUtils.handleUncaughtException( jobTerminationFuture, @@ -623,13 +633,27 @@ private void runJob(JobManagerRunner jobManagerRunner, ExecutionType executionTy registerJobManagerRunnerTerminationFuture(jobId, jobTerminationFuture); } - private CleanupJobState handleJobManagerRunnerResult( + @Nullable + private Void logCleanupErrorWarning(JobID jobId, Throwable cleanupError) { + log.warn( + "The cleanup of job {} failed. The job's artifacts in the different directories ('{}', '{}', '{}') and its JobResultStore entry in '{}' (in HA mode) should be checked for manual cleanup.", + jobId, + configuration.get(HighAvailabilityOptions.HA_STORAGE_PATH), + configuration.get(BlobServerOptions.STORAGE_DIRECTORY), + configuration.get(CheckpointingOptions.CHECKPOINTS_DIRECTORY), + configuration.get(JobResultStoreOptions.STORAGE_PATH), + cleanupError); + return null; + } + + private CompletableFuture handleJobManagerRunnerResult( JobManagerRunnerResult jobManagerRunnerResult, ExecutionType executionType) { if (jobManagerRunnerResult.isInitializationFailure() && executionType == ExecutionType.RECOVERY) { - return jobManagerRunnerFailed( - jobManagerRunnerResult.getExecutionGraphInfo().getJobId(), - jobManagerRunnerResult.getInitializationFailure()); + return CompletableFuture.completedFuture( + jobManagerRunnerFailed( + jobManagerRunnerResult.getExecutionGraphInfo().getJobId(), + jobManagerRunnerResult.getInitializationFailure())); } return jobReachedTerminalState(jobManagerRunnerResult.getExecutionGraphInfo()); } @@ -964,7 +988,7 @@ private CompletableFuture removeJob(JobID jobId, CleanupJobState cleanupJo case GLOBAL: return globalResourceCleaner .cleanupAsync(jobId) - .thenRun(() -> markJobAsClean(jobId)); + .thenRunAsync(() -> markJobAsClean(jobId), ioExecutor); default: throw new IllegalStateException("Invalid cleanup state: " + cleanupJobState); } @@ -1009,7 +1033,8 @@ protected void onFatalError(Throwable throwable) { fatalErrorHandler.onFatalError(throwable); } - protected CleanupJobState jobReachedTerminalState(ExecutionGraphInfo executionGraphInfo) { + protected CompletableFuture jobReachedTerminalState( + ExecutionGraphInfo executionGraphInfo) { final ArchivedExecutionGraph archivedExecutionGraph = executionGraphInfo.getArchivedExecutionGraph(); final JobStatus terminalJobStatus = archivedExecutionGraph.getState(); @@ -1041,35 +1066,50 @@ protected CleanupJobState jobReachedTerminalState(ExecutionGraphInfo executionGr archiveExecutionGraph(executionGraphInfo); - if (terminalJobStatus.isGloballyTerminalState()) { - final JobID jobId = executionGraphInfo.getJobId(); - try { - if (jobResultStore.hasCleanJobResultEntry(jobId)) { - log.warn( - "Job {} is already marked as clean but clean up was triggered again.", - jobId); - } else if (!jobResultStore.hasDirtyJobResultEntry(jobId)) { - jobResultStore.createDirtyResult( - new JobResultEntry( - JobResult.createFrom( - executionGraphInfo.getArchivedExecutionGraph()))); - log.info( - "Job {} has been registered for cleanup in the JobResultStore after reaching a terminal state.", - jobId); - } - } catch (IOException e) { - fatalErrorHandler.onFatalError( - new FlinkException( - String.format( - "The job %s couldn't be marked as pre-cleanup finished in JobResultStore.", - jobId), - e)); - } + if (!terminalJobStatus.isGloballyTerminalState()) { + return CompletableFuture.completedFuture(CleanupJobState.LOCAL); } - return terminalJobStatus.isGloballyTerminalState() - ? CleanupJobState.GLOBAL - : CleanupJobState.LOCAL; + final CompletableFuture writeFuture = new CompletableFuture<>(); + final JobID jobId = executionGraphInfo.getJobId(); + + ioExecutor.execute( + () -> { + try { + if (jobResultStore.hasCleanJobResultEntry(jobId)) { + log.warn( + "Job {} is already marked as clean but clean up was triggered again.", + jobId); + } else if (!jobResultStore.hasDirtyJobResultEntry(jobId)) { + jobResultStore.createDirtyResult( + new JobResultEntry( + JobResult.createFrom( + executionGraphInfo + .getArchivedExecutionGraph()))); + log.info( + "Job {} has been registered for cleanup in the JobResultStore after reaching a terminal state.", + jobId); + } + } catch (IOException e) { + writeFuture.completeExceptionally(e); + return; + } + writeFuture.complete(null); + }); + + return writeFuture.handleAsync( + (ignored, error) -> { + if (error != null) { + fatalErrorHandler.onFatalError( + new FlinkException( + String.format( + "The job %s couldn't be marked as pre-cleanup finished in JobResultStore.", + executionGraphInfo.getJobId()), + error)); + } + return CleanupJobState.GLOBAL; + }, + getMainThreadExecutor()); } private void archiveExecutionGraph(ExecutionGraphInfo executionGraphInfo) { @@ -1194,15 +1234,10 @@ private CompletableFuture waitForTerminatingJob( getMainThreadExecutor()); } + @VisibleForTesting CompletableFuture getJobTerminationFuture(JobID jobId) { - if (jobManagerRunnerRegistry.isRegistered(jobId)) { - return FutureUtils.completedExceptionally( - new DispatcherException( - String.format("Job with job id %s is still running.", jobId))); - } else { - return jobManagerRunnerTerminationFutures.getOrDefault( - jobId, CompletableFuture.completedFuture(null)); - } + return jobManagerRunnerTerminationFutures.getOrDefault( + jobId, CompletableFuture.completedFuture(null)); } private void registerDispatcherMetrics(MetricGroup jobManagerMetricGroup) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java index 76afe73bf9d18..85a08332e6653 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java @@ -129,26 +129,33 @@ public CompletableFuture cancelJob(JobID jobId, Time timeout) { } @Override - protected CleanupJobState jobReachedTerminalState(ExecutionGraphInfo executionGraphInfo) { + protected CompletableFuture jobReachedTerminalState( + ExecutionGraphInfo executionGraphInfo) { final ArchivedExecutionGraph archivedExecutionGraph = executionGraphInfo.getArchivedExecutionGraph(); - final CleanupJobState cleanupHAState = super.jobReachedTerminalState(executionGraphInfo); - - JobStatus jobStatus = - Objects.requireNonNull( - archivedExecutionGraph.getState(), "JobStatus should not be null here."); - if (jobStatus.isGloballyTerminalState() - && (jobCancelled || executionMode == ClusterEntrypoint.ExecutionMode.DETACHED)) { - // shut down if job is cancelled or we don't have to wait for the execution result - // retrieval - log.info( - "Shutting down cluster with state {}, jobCancelled: {}, executionMode: {}", - jobStatus, - jobCancelled, - executionMode); - shutDownFuture.complete(ApplicationStatus.fromJobStatus(jobStatus)); - } + final CompletableFuture cleanupHAState = + super.jobReachedTerminalState(executionGraphInfo); + + return cleanupHAState.thenApply( + cleanupJobState -> { + JobStatus jobStatus = + Objects.requireNonNull( + archivedExecutionGraph.getState(), + "JobStatus should not be null here."); + if (jobStatus.isGloballyTerminalState() + && (jobCancelled + || executionMode == ClusterEntrypoint.ExecutionMode.DETACHED)) { + // shut down if job is cancelled or we don't have to wait for the execution + // result retrieval + log.info( + "Shutting down cluster with state {}, jobCancelled: {}, executionMode: {}", + jobStatus, + jobCancelled, + executionMode); + shutDownFuture.complete(ApplicationStatus.fromJobStatus(jobStatus)); + } - return cleanupHAState; + return cleanupJobState; + }); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java index 4ae76780a4f7d..e657c8d740012 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java @@ -30,6 +30,7 @@ import org.apache.flink.runtime.dispatcher.JobCancellationFailedException; import org.apache.flink.runtime.dispatcher.UnavailableDispatcherOperationException; import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.jobmaster.JobManagerRunner; import org.apache.flink.runtime.jobmaster.JobManagerRunnerResult; import org.apache.flink.runtime.jobmaster.JobMaster; @@ -129,7 +130,7 @@ private void cleanupCheckpoints() throws Exception { } try { - checkpointIDCounter.shutdown(getJobStatus()); + checkpointIDCounter.shutdown(getJobStatus()).get(); } catch (Exception e) { exception = ExceptionUtils.firstOrSuppressed(e, exception); } @@ -145,7 +146,11 @@ private CompletedCheckpointStore createCompletedCheckpointStore() throws Excepti DefaultCompletedCheckpointStoreUtils.getMaximumNumberOfRetainedCheckpoints( jobManagerConfiguration, LOG), sharedStateRegistryFactory, - cleanupExecutor); + cleanupExecutor, + // Using RestoreMode.CLAIM to be able to discard shared state, if any. + // Note that it also means that the original shared state might be discarded as well + // because the initial checkpoint might be subsumed. + RestoreMode.CLAIM); } private CheckpointIDCounter createCheckpointIDCounter() throws Exception { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactory.java index 79557b0a6adca..1ec88a100a5f2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactory.java @@ -78,9 +78,7 @@ private static ExponentialBackoffRetryStrategy createExponentialBackoffRetryStra final Duration maxDelay = configuration.get(CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_BACKOFF); final int maxAttempts = - configuration.getInteger( - CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_ATTEMPTS, - Integer.MAX_VALUE); + configuration.get(CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_ATTEMPTS); return new ExponentialBackoffRetryStrategy(maxAttempts, minDelay, maxDelay); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java index 6f9d0bf2fd377..63187002acce4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java @@ -147,7 +147,7 @@ private Collection recoverJobs(Set recoveredDirtyJobResults) { for (JobID jobId : jobIds) { if (!recoveredDirtyJobResults.contains(jobId)) { - recoveredJobGraphs.add(recoverJob(jobId)); + tryRecoverJob(jobId).ifPresent(recoveredJobGraphs::add); } } @@ -164,10 +164,16 @@ private Collection getJobIds() { } } - private JobGraph recoverJob(JobID jobId) { + private Optional tryRecoverJob(JobID jobId) { log.info("Trying to recover job with job id {}.", jobId); try { - return jobGraphStore.recoverJobGraph(jobId); + final JobGraph jobGraph = jobGraphStore.recoverJobGraph(jobId); + if (jobGraph == null) { + log.info( + "Skipping recovery of job with job id {}, because it already finished in a previous execution", + jobId); + } + return Optional.ofNullable(jobGraph); } catch (Exception e) { throw new FlinkRuntimeException( String.format("Could not recover job with job id %s.", jobId), e); @@ -264,7 +270,7 @@ private DispatcherGateway getDispatcherGatewayInternal() { } private Optional recoverJobIfRunning(JobID jobId) { - return supplyUnsynchronizedIfRunning(() -> recoverJob(jobId)); + return supplyUnsynchronizedIfRunning(() -> tryRecoverJob(jobId)).flatMap(x -> x); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java index 352a184d62f42..487d4e3da77e6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java @@ -288,6 +288,10 @@ public boolean tryAssignResource(final LogicalSlot logicalSlot) { && !taskManagerLocationFuture.isDone()) { taskManagerLocationFuture.complete(logicalSlot.getTaskManagerLocation()); assignedAllocationID = logicalSlot.getAllocationId(); + getVertex() + .setLatestPriorSlotAllocation( + assignedResource.getTaskManagerLocation(), + logicalSlot.getAllocationId()); return true; } else { // free assigned resource and return false diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java index fef96e27f10ed..4bf20f3faabcf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java @@ -36,6 +36,7 @@ import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.util.EvictingBoundedList; +import org.apache.flink.util.Preconditions; import javax.annotation.Nullable; @@ -47,7 +48,6 @@ import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; -import java.util.function.Function; import static org.apache.flink.runtime.execution.ExecutionState.FINISHED; import static org.apache.flink.util.Preconditions.checkArgument; @@ -84,6 +84,11 @@ public class ExecutionVertex private final ArrayList inputSplits; + /** This field holds the allocation id of the last successful assignment. */ + @Nullable private TaskManagerLocation lastAssignedLocation; + + @Nullable private AllocationID lastAssignedAllocationID; + // -------------------------------------------------------------------------------------------- /** @@ -280,44 +285,23 @@ public ArchivedExecution getPriorExecutionAttempt(int attemptNumber) { } } - /** - * Gets the latest property from a prior execution that is not null. - * - * @param extractor defining the property to extract - * @param type of the property - * @return Optional containing the latest property if it exists; otherwise {@code - * Optional.empty()}. - */ - private Optional getLatestPriorProperty(Function extractor) { - int index = priorExecutions.size() - 1; - - while (index >= 0 && !priorExecutions.isDroppedIndex(index)) { - final ArchivedExecution archivedExecution = priorExecutions.get(index); - - final T extractedValue = extractor.apply(archivedExecution); - - if (extractedValue != null) { - return Optional.of(extractedValue); - } - - index -= 1; - } - - return Optional.empty(); + void setLatestPriorSlotAllocation( + TaskManagerLocation taskManagerLocation, AllocationID lastAssignedAllocationID) { + this.lastAssignedLocation = Preconditions.checkNotNull(taskManagerLocation); + this.lastAssignedAllocationID = Preconditions.checkNotNull(lastAssignedAllocationID); } /** - * Gets the location where the latest completed/canceled/failed execution of the vertex's task - * happened. + * Gets the location that an execution of this vertex was assigned to. * - * @return The latest prior execution location, or null, if there is none, yet. + * @return The last execution location, or null, if there is none, yet. */ - public Optional findLatestPriorLocation() { - return getLatestPriorProperty(ArchivedExecution::getAssignedResourceLocation); + public Optional findLastLocation() { + return Optional.ofNullable(lastAssignedLocation); } - public Optional findLatestPriorAllocation() { - return getLatestPriorProperty(ArchivedExecution::getAssignedAllocationID); + public Optional findLastAllocation() { + return Optional.ofNullable(lastAssignedAllocationID); } EvictingBoundedList getCopyOfPriorExecutionsList() { @@ -353,7 +337,7 @@ public Optional getPreferredLocationBasedOnState() { // only restore to same execution if it has state if (currentExecution.getTaskRestore() != null && currentExecution.getTaskRestore().getTaskStateSnapshot().hasState()) { - return findLatestPriorLocation(); + return findLastLocation(); } return Optional.empty(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java index a7ed3c5d4fc53..010ce77e74cc7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rest.messages.json.JobResultDeserializer; import org.apache.flink.runtime.rest.messages.json.JobResultSerializer; +import org.apache.flink.runtime.util.NonClosingOutputStreamDecorator; import org.apache.flink.util.Preconditions; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; @@ -114,7 +115,7 @@ public static String createDefaultJobResultStorePath(String baseDir, String clus * @return A path for a dirty entry for the given the Job ID. */ private Path constructDirtyPath(JobID jobId) { - return new Path(this.basePath.getPath(), jobId.toString() + DIRTY_FILE_EXTENSION); + return constructEntryPath(jobId.toString() + DIRTY_FILE_EXTENSION); } /** @@ -125,15 +126,23 @@ private Path constructDirtyPath(JobID jobId) { * @return A path for a clean entry for the given the Job ID. */ private Path constructCleanPath(JobID jobId) { - return new Path(this.basePath.getPath(), jobId.toString() + ".json"); + return constructEntryPath(jobId.toString() + FILE_EXTENSION); + } + + @VisibleForTesting + Path constructEntryPath(String fileName) { + return new Path(this.basePath, fileName); } @Override public void createDirtyResultInternal(JobResultEntry jobResultEntry) throws IOException { final Path path = constructDirtyPath(jobResultEntry.getJobId()); try (OutputStream os = fileSystem.create(path, FileSystem.WriteMode.NO_OVERWRITE)) { - mapper.writeValue(os, new JsonJobResultEntry(jobResultEntry)); - os.flush(); + mapper.writeValue( + // working around the internally used _writeAndClose method to ensure that close + // is only called once + new NonClosingOutputStreamDecorator(os), + new JsonJobResultEntry(jobResultEntry)); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedHaServicesWithLeadershipControl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedHaServicesWithLeadershipControl.java index ce59d0d99e926..b9859ba0f2718 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedHaServicesWithLeadershipControl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedHaServicesWithLeadershipControl.java @@ -40,13 +40,18 @@ public EmbeddedHaServicesWithLeadershipControl(Executor executor) { this( executor, new PerJobCheckpointRecoveryFactory( - (maxCheckpoints, previous, stateRegistryFactory, ioExecutor) -> { + (maxCheckpoints, + previous, + stateRegistryFactory, + ioExecutor, + restoreMode) -> { List checkpoints = previous != null ? previous.getAllCheckpoints() : Collections.emptyList(); SharedStateRegistry stateRegistry = - stateRegistryFactory.create(ioExecutor, checkpoints); + stateRegistryFactory.create( + ioExecutor, checkpoints, restoreMode); if (previous != null) { if (!previous.getShutdownStatus().isPresent()) { throw new IllegalStateException( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/SubtaskStateMapper.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/SubtaskStateMapper.java index a2fc87b371904..c9136cebc549b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/SubtaskStateMapper.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/SubtaskStateMapper.java @@ -89,7 +89,7 @@ public boolean isAmbiguous() { *

Example:
* old assignment: 0 -> [0;43); 1 -> [43;87); 2 -> [87;128)
* new assignment: 0 -> [0;64]; 1 -> [64;128)
- * subtask 0 recovers data from old subtask 0 + 1 and subtask 1 recovers data from old subtask 0 + * subtask 0 recovers data from old subtask 0 + 1 and subtask 1 recovers data from old subtask 1 * + 2 * *

For all downscale from n to [n-1 .. n/2], each new subtasks get exactly two old subtasks diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/RestoreMode.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/RestoreMode.java index 10a4f2ac60d1e..da6c325265a0d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/RestoreMode.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/RestoreMode.java @@ -53,4 +53,6 @@ public enum RestoreMode implements DescribedEnum { public InlineElement getDescription() { return text(description); } + + public static final RestoreMode DEFAULT = NO_CLAIM; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SavepointConfigOptions.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SavepointConfigOptions.java index e8a9dd86d1c70..a38e05c5f3f5e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SavepointConfigOptions.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SavepointConfigOptions.java @@ -52,7 +52,7 @@ public class SavepointConfigOptions { public static final ConfigOption RESTORE_MODE = key("execution.savepoint-restore-mode") .enumType(RestoreMode.class) - .defaultValue(RestoreMode.NO_CLAIM) + .defaultValue(RestoreMode.DEFAULT) .withDescription( "Describes the mode how Flink should restore from the given" + " savepoint or retained checkpoint."); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStore.java index fda4964c31f88..e66a1729f82f0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStore.java @@ -249,12 +249,10 @@ public CompletableFuture globalCleanupAsync(JobID jobId, Executor executor () -> { LOG.debug("Removing job graph {} from {}.", jobId, jobGraphStateHandleStore); - if (addedJobGraphs.contains(jobId)) { - final String name = jobGraphStoreUtil.jobIDToName(jobId); - releaseAndRemoveOrThrowCompletionException(jobId, name); + final String name = jobGraphStoreUtil.jobIDToName(jobId); + releaseAndRemoveOrThrowCompletionException(jobId, name); - addedJobGraphs.remove(jobId); - } + addedJobGraphs.remove(jobId); LOG.info("Removed job graph {} from {}.", jobId, jobGraphStateHandleStore); }, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/DescriptiveStatisticsHistogramStatistics.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/DescriptiveStatisticsHistogramStatistics.java index 12975b69d2d7d..8eb5036e64dd6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/DescriptiveStatisticsHistogramStatistics.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/DescriptiveStatisticsHistogramStatistics.java @@ -17,6 +17,7 @@ package org.apache.flink.runtime.metrics; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.metrics.HistogramStatistics; import org.apache.commons.math3.exception.MathIllegalArgumentException; @@ -90,15 +91,16 @@ public long getMin() { * will not return a value but instead populate this class so that further values can be * retrieved from it. */ - private static class CommonMetricsSnapshot implements UnivariateStatistic, Serializable { - private static final long serialVersionUID = 1L; + @VisibleForTesting + static class CommonMetricsSnapshot implements UnivariateStatistic, Serializable { + private static final long serialVersionUID = 2L; - private long count = 0; + private double[] data; private double min = Double.NaN; private double max = Double.NaN; private double mean = Double.NaN; private double stddev = Double.NaN; - private Percentile percentilesImpl = new Percentile().withNaNStrategy(NaNStrategy.FIXED); + private transient Percentile percentilesImpl; @Override public double evaluate(final double[] values) throws MathIllegalArgumentException { @@ -108,8 +110,7 @@ public double evaluate(final double[] values) throws MathIllegalArgumentExceptio @Override public double evaluate(double[] values, int begin, int length) throws MathIllegalArgumentException { - this.count = length; - percentilesImpl.setData(values, begin, length); + this.data = values; SimpleStats secondMoment = new SimpleStats(); secondMoment.evaluate(values, begin, length); @@ -125,17 +126,16 @@ public double evaluate(double[] values, int begin, int length) @Override public CommonMetricsSnapshot copy() { CommonMetricsSnapshot result = new CommonMetricsSnapshot(); - result.count = count; + result.data = Arrays.copyOf(data, data.length); result.min = min; result.max = max; result.mean = mean; result.stddev = stddev; - result.percentilesImpl = percentilesImpl.copy(); return result; } long getCount() { - return count; + return data.length; } double getMin() { @@ -155,12 +155,23 @@ long getCount() { } double getPercentile(double p) { + maybeInitPercentile(); return percentilesImpl.evaluate(p); } double[] getValues() { + maybeInitPercentile(); return percentilesImpl.getData(); } + + private void maybeInitPercentile() { + if (percentilesImpl == null) { + percentilesImpl = new Percentile().withNaNStrategy(NaNStrategy.FIXED); + } + if (data != null) { + percentilesImpl.setData(data); + } + } } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/persistence/StateHandleStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/persistence/StateHandleStore.java index 5cbfeac741ec6..6c9f521bd50f7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/persistence/StateHandleStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/persistence/StateHandleStore.java @@ -117,11 +117,11 @@ void replace(String name, R resourceVersion, T state) /** * Releases the lock for the given state handle and tries to remove the state handle if it is no - * longer locked. It returns the {@link RetrievableStateHandle} stored under the given state - * node if any. Also the state on the external storage will be discarded. + * longer locked. Also the state on the external storage will be discarded. * * @param name Key name in ConfigMap or child path name in ZooKeeper - * @return True if the state handle could be removed. + * @return {@code true} if the state handle is removed (also if it didn't exist in the first + * place); otherwise {@code false}. * @throws Exception if releasing, removing the handles or discarding the state failed */ boolean releaseAndTryRemove(String name) throws Exception; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/async/OperationKey.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/async/OperationKey.java index 85df2ac19f467..d908eef6b858a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/async/OperationKey.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/async/OperationKey.java @@ -21,13 +21,16 @@ import org.apache.flink.runtime.rest.messages.TriggerId; import org.apache.flink.util.Preconditions; +import java.io.Serializable; import java.util.Objects; /** * Any operation key for the {@link AbstractAsynchronousOperationHandlers} must extend this class. * It is used to store the trigger id. */ -public class OperationKey { +public class OperationKey implements Serializable { + + private static final long serialVersionUID = 6138473450686379255L; private final TriggerId triggerId; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/AsynchronousJobOperationKey.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/AsynchronousJobOperationKey.java index 2512a94af011d..7bee2953a23de 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/AsynchronousJobOperationKey.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/AsynchronousJobOperationKey.java @@ -37,6 +37,8 @@ @Immutable public class AsynchronousJobOperationKey extends OperationKey { + private static final long serialVersionUID = -4907777251835275859L; + private final JobID jobId; private AsynchronousJobOperationKey(final TriggerId triggerId, final JobID jobId) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java index 3af4b61ece238..e9e369c3d1915 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java @@ -732,7 +732,7 @@ public ResourceProfile getResourceProfile(final ExecutionVertexID executionVerte @Override public Optional findPriorAllocationId( final ExecutionVertexID executionVertexId) { - return getExecutionVertex(executionVertexId).findLatestPriorAllocation(); + return getExecutionVertex(executionVertexId).findLastAllocation(); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java index c997cc2f1098f..0ca22335c078c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java @@ -245,7 +245,7 @@ private void shutDownCheckpointServices(JobStatus jobStatus) { } try { - checkpointIdCounter.shutdown(jobStatus); + checkpointIdCounter.shutdown(jobStatus).get(); } catch (Exception e) { exception = ExceptionUtils.firstOrSuppressed(e, exception); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerUtils.java index 6b14801d8ebe5..87f2e56c6ba15 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerUtils.java @@ -30,6 +30,7 @@ import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.executiongraph.DefaultExecutionGraphBuilder; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.state.SharedStateRegistry; import org.slf4j.Logger; @@ -55,7 +56,12 @@ public static CompletedCheckpointStore createCompletedCheckpointStoreIfCheckpoin if (DefaultExecutionGraphBuilder.isCheckpointingEnabled(jobGraph)) { try { return createCompletedCheckpointStore( - configuration, checkpointRecoveryFactory, ioExecutor, log, jobId); + configuration, + checkpointRecoveryFactory, + ioExecutor, + log, + jobId, + jobGraph.getSavepointRestoreSettings().getRestoreMode()); } catch (Exception e) { throw new JobExecutionException( jobId, @@ -73,14 +79,16 @@ static CompletedCheckpointStore createCompletedCheckpointStore( CheckpointRecoveryFactory recoveryFactory, Executor ioExecutor, Logger log, - JobID jobId) + JobID jobId, + RestoreMode restoreMode) throws Exception { return recoveryFactory.createRecoveredCompletedCheckpointStore( jobId, DefaultCompletedCheckpointStoreUtils.getMaximumNumberOfRetainedCheckpoints( jobManagerConfig, log), SharedStateRegistry.DEFAULT_FACTORY, - ioExecutor); + ioExecutor, + restoreMode); } public static CheckpointIDCounter createCheckpointIDCounterIfCheckpointingIsEnabled( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java index 999b2afb5532d..562c17f05c32a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java @@ -463,7 +463,7 @@ private void stopCheckpointServicesSafely(JobStatus terminalState) { } try { - checkpointIdCounter.shutdown(terminalState); + checkpointIdCounter.shutdown(terminalState).get(); } catch (Exception e) { exception = ExceptionUtils.firstOrSuppressed(e, exception); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/StopWithSavepoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/StopWithSavepoint.java index 2c03e7454f1ac..de4584e004243 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/StopWithSavepoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/StopWithSavepoint.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.scheduler.ExecutionGraphHandler; import org.apache.flink.runtime.scheduler.OperatorCoordinatorHandler; import org.apache.flink.runtime.scheduler.exceptionhistory.ExceptionHistoryEntry; +import org.apache.flink.runtime.scheduler.stopwithsavepoint.StopWithSavepointStoppingException; import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; import org.apache.flink.util.concurrent.FutureUtils; @@ -146,7 +147,16 @@ public JobStatus getJobStatus() { @Override void onFailure(Throwable cause) { operationFailureCause = cause; - FailureResultUtil.restartOrFail(context.howToHandleFailure(cause), context, this); + if (savepoint == null) { + FailureResultUtil.restartOrFail(context.howToHandleFailure(cause), context, this); + } else { + // savepoint has been create successfully, but the job failed while committing side + // effects + final StopWithSavepointStoppingException ex = + new StopWithSavepointStoppingException(savepoint, this.getJobId(), cause); + this.operationFuture.completeExceptionally(ex); + FailureResultUtil.restartOrFail(context.howToHandleFailure(ex), context, this); + } } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointStoppingException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointStoppingException.java new file mode 100644 index 0000000000000..5accea9a02d05 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointStoppingException.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.scheduler.stopwithsavepoint; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.throwable.ThrowableAnnotation; +import org.apache.flink.runtime.throwable.ThrowableType; +import org.apache.flink.util.FlinkException; + +/** + * Exception thrown when a savepoint has been created successfully when stopping with savepoint, but + * the job has not finished. In that case side-effects might have not been committed. This exception + * is used to communicate that to the use. + */ +@Experimental +@ThrowableAnnotation(ThrowableType.NonRecoverableError) +public class StopWithSavepointStoppingException extends FlinkException { + private final String savepointPath; + + public StopWithSavepointStoppingException(String savepointPath, JobID jobID) { + super(formatMessage(savepointPath, jobID)); + this.savepointPath = savepointPath; + } + + public StopWithSavepointStoppingException(String savepointPath, JobID jobID, Throwable cause) { + super(formatMessage(savepointPath, jobID), cause); + this.savepointPath = savepointPath; + } + + private static String formatMessage(String savepointPath, JobID jobID) { + return String.format( + "A savepoint has been created at: %s, but the corresponding job %s failed " + + "during stopping. The savepoint is consistent, but might have " + + "uncommitted transactions. If you want to commit the transaction " + + "please restart a job from this savepoint.", + savepointPath, jobID); + } + + public String getSavepointPath() { + return savepointPath; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandlerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandlerImpl.java index aec32d34981f7..f3416135656de 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandlerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandlerImpl.java @@ -24,7 +24,6 @@ import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.scheduler.SchedulerNG; -import org.apache.flink.util.FlinkException; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; @@ -167,16 +166,13 @@ private void handleAnyExecutionNotFinished(Set notFinishedExecut */ private void terminateExceptionallyWithGlobalFailover( Iterable unfinishedExecutionStates, String savepointPath) { - String errorMessage = - String.format( - "Inconsistent execution state after stopping with savepoint. At least one execution is still in one of the following states: %s. A global fail-over is triggered to recover the job %s.", - StringUtils.join(unfinishedExecutionStates, ", "), jobId); - FlinkException inconsistentFinalStateException = new FlinkException(errorMessage); + StopWithSavepointStoppingException inconsistentFinalStateException = + new StopWithSavepointStoppingException(savepointPath, jobId); log.warn( - "A savepoint was created at {} but the corresponding job {} didn't terminate successfully.", - savepointPath, - jobId, + "Inconsistent execution state after stopping with savepoint. At least one" + + " execution is still in one of the following states: {}.", + StringUtils.join(unfinishedExecutionStates, ", "), inconsistentFinalStateException); scheduler.handleGlobalFailure(inconsistentFinalStateException); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java index 5e560fad7f2cf..4434a18b5dc56 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java @@ -195,14 +195,17 @@ public void assignSplits(SplitsAssignment assignment) { callInCoordinatorThread( () -> { // Ensure all the subtasks in the assignment have registered. - for (Integer subtaskId : assignment.assignment().keySet()) { - if (!registeredReaders.containsKey(subtaskId)) { - throw new IllegalArgumentException( - String.format( - "Cannot assign splits %s to subtask %d because the subtask is not registered.", - registeredReaders.get(subtaskId), subtaskId)); - } - } + assignment + .assignment() + .forEach( + (id, splits) -> { + if (!registeredReaders.containsKey(id)) { + throw new IllegalArgumentException( + String.format( + "Cannot assign splits %s to subtask %d because the subtask is not registered.", + splits, id)); + } + }); assignmentTracker.recordSplitAssignment(assignment); assignment diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/InternalPriorityQueue.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/InternalPriorityQueue.java index 25816eb49b4d8..17bdd6ad35fe6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/InternalPriorityQueue.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/InternalPriorityQueue.java @@ -39,6 +39,9 @@ public interface InternalPriorityQueue { * Retrieves and removes the first element (w.r.t. the order) of this set, or returns {@code * null} if this set is empty. * + *

NOTE: Correct key (i.e. the key of the polled element) must be set on KeyContext before + * calling this method. + * * @return the first element of this ordered set, or {@code null} if this set is empty. */ @Nullable @@ -67,6 +70,9 @@ public interface InternalPriorityQueue { /** * Removes the given element from the set, if is contained in the set. * + *

NOTE: Correct key (i.e. the key of the polled element) must be set on KeyContext before + * calling this method. + * * @param toRemove the element to remove. * @return true if the operation changed the head element or if it is unclear if * the head element changed. Only returns false if the head element was not diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupRangeOffsets.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupRangeOffsets.java index 0d9046a66d582..7ba4ac8f63d56 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupRangeOffsets.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupRangeOffsets.java @@ -216,8 +216,7 @@ public String toString() { return "KeyGroupRangeOffsets{" + "keyGroupRange=" + keyGroupRange - + ", offsets=" - + Arrays.toString(offsets) + + (offsets.length > 10 ? "" : ", offsets=" + Arrays.toString(offsets)) + '}'; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java index 7172bec4c243c..b816f09e767ae 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.state; import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; +import org.apache.flink.runtime.jobgraph.RestoreMode; /** * This registry manages state that is shared across (incremental) checkpoints, and is responsible @@ -32,11 +33,11 @@ public interface SharedStateRegistry extends AutoCloseable { /** A singleton object for the default implementation of a {@link SharedStateRegistryFactory} */ SharedStateRegistryFactory DEFAULT_FACTORY = - (deleteExecutor, checkpoints) -> { + (deleteExecutor, checkpoints, restoreMode) -> { SharedStateRegistry sharedStateRegistry = new SharedStateRegistryImpl(deleteExecutor); for (CompletedCheckpoint checkpoint : checkpoints) { - checkpoint.registerSharedStatesAfterRestored(sharedStateRegistry); + checkpoint.registerSharedStatesAfterRestored(sharedStateRegistry, restoreMode); } return sharedStateRegistry; }; @@ -66,10 +67,25 @@ StreamStateHandle registerReference( /** * Register given shared states in the registry. * + *

NOTE: For state from checkpoints from other jobs or runs (i.e. after recovery), please use + * {@link #registerAllAfterRestored(CompletedCheckpoint, RestoreMode)} + * * @param stateHandles The shared states to register. * @param checkpointID which uses the states. */ void registerAll(Iterable stateHandles, long checkpointID); + /** + * Set the lowest checkpoint ID below which no state is discarded, inclusive. + * + *

After recovery from an incremental checkpoint, its state should NOT be discarded, even if + * {@link #unregisterUnusedState(long) not used} anymore (unless recovering in {@link + * RestoreMode#CLAIM CLAIM} mode). + * + *

This should hold for both cases: when recovering from that initial checkpoint; and from + * any subsequent checkpoint derived from it. + */ + void registerAllAfterRestored(CompletedCheckpoint checkpoint, RestoreMode mode); + void checkpointCompleted(long checkpointId); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryFactory.java index bbdd2fd095944..bc8118cce4286 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryFactory.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.state; import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; +import org.apache.flink.runtime.jobgraph.RestoreMode; import java.util.Collection; import java.util.concurrent.Executor; @@ -29,10 +30,13 @@ public interface SharedStateRegistryFactory { /** * Factory method for {@link SharedStateRegistry}. * - * @param checkpoints whose shared state will be registered. * @param deleteExecutor executor used to run (async) deletes. + * @param checkpoints whose shared state will be registered. + * @param restoreMode the mode in which the given checkpoints were restored * @return a SharedStateRegistry object */ SharedStateRegistry create( - Executor deleteExecutor, Collection checkpoints); + Executor deleteExecutor, + Collection checkpoints, + RestoreMode restoreMode); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java index b87d864608646..4dce645527722 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java @@ -19,6 +19,8 @@ package org.apache.flink.runtime.state; import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.util.concurrent.Executors; import org.slf4j.Logger; @@ -51,6 +53,9 @@ public class SharedStateRegistryImpl implements SharedStateRegistry { /** Executor for async state deletion */ private final Executor asyncDisposalExecutor; + /** Checkpoint ID below which no state is discarded, inclusive. */ + private long highestNotClaimedCheckpointID = -1L; + /** Default uses direct executor to delete unreferenced state */ public SharedStateRegistryImpl() { this(Executors.directExecutor()); @@ -147,7 +152,9 @@ public void unregisterUnusedState(long lowestCheckpointID) { while (it.hasNext()) { SharedStateEntry entry = it.next(); if (entry.lastUsedCheckpointID < lowestCheckpointID) { - subsumed.add(entry.stateHandle); + if (entry.createdByCheckpointID > highestNotClaimedCheckpointID) { + subsumed.add(entry.stateHandle); + } it.remove(); } } @@ -174,6 +181,20 @@ public void registerAll( } } + @Override + public void registerAllAfterRestored(CompletedCheckpoint checkpoint, RestoreMode mode) { + registerAll(checkpoint.getOperatorStates().values(), checkpoint.getCheckpointID()); + // In NO_CLAIM and LEGACY restore modes, shared state of the initial checkpoints must be + // preserved. This is achieved by advancing highestRetainCheckpointID here, and then + // checking entry.createdByCheckpointID against it on checkpoint subsumption. + // In CLAIM restore mode, the shared state of the initial checkpoints must be + // discarded as soon as it becomes unused - so highestRetainCheckpointID is not updated. + if (mode != RestoreMode.CLAIM) { + highestNotClaimedCheckpointID = + Math.max(highestNotClaimedCheckpointID, checkpoint.getCheckpointID()); + } + } + @Override public void checkpointCompleted(long checkpointId) { for (SharedStateEntry entry : registeredStates.values()) { @@ -251,6 +272,8 @@ private static final class SharedStateEntry { /** The shared state handle */ StreamStateHandle stateHandle; + private final long createdByCheckpointID; + private long lastUsedCheckpointID; /** Whether this entry is included into a confirmed checkpoint. */ @@ -258,6 +281,7 @@ private static final class SharedStateEntry { SharedStateEntry(StreamStateHandle value, long checkpointID) { this.stateHandle = value; + this.createdByCheckpointID = checkpointID; this.lastUsedCheckpointID = checkpointID; } @@ -266,6 +290,8 @@ public String toString() { return "SharedStateEntry{" + "stateHandle=" + stateHandle + + ", createdByCheckpointID=" + + createdByCheckpointID + ", lastUsedCheckpointID=" + lastUsedCheckpointID + '}'; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SnappyStreamCompressionDecorator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SnappyStreamCompressionDecorator.java index 5b024149d1f85..ff3318b55cb58 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SnappyStreamCompressionDecorator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SnappyStreamCompressionDecorator.java @@ -20,7 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.runtime.util.NonClosingInputStreamDecorator; -import org.apache.flink.runtime.util.NonClosingOutpusStreamDecorator; +import org.apache.flink.runtime.util.NonClosingOutputStreamDecorator; import org.xerial.snappy.SnappyFramedInputStream; import org.xerial.snappy.SnappyFramedOutputStream; @@ -42,7 +42,7 @@ public class SnappyStreamCompressionDecorator extends StreamCompressionDecorator private static final double MIN_COMPRESSION_RATIO = 0.85d; @Override - protected OutputStream decorateWithCompression(NonClosingOutpusStreamDecorator stream) + protected OutputStream decorateWithCompression(NonClosingOutputStreamDecorator stream) throws IOException { return new SnappyFramedOutputStream(stream, COMPRESSION_BLOCK_SIZE, MIN_COMPRESSION_RATIO); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StreamCompressionDecorator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StreamCompressionDecorator.java index ad9e52c64e276..e32a51b97bbc4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StreamCompressionDecorator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StreamCompressionDecorator.java @@ -20,7 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.runtime.util.NonClosingInputStreamDecorator; -import org.apache.flink.runtime.util.NonClosingOutpusStreamDecorator; +import org.apache.flink.runtime.util.NonClosingOutputStreamDecorator; import java.io.IOException; import java.io.InputStream; @@ -46,7 +46,7 @@ public abstract class StreamCompressionDecorator implements Serializable { * @return an output stream that is decorated by the compression scheme. */ public final OutputStream decorateWithCompression(OutputStream stream) throws IOException { - return decorateWithCompression(new NonClosingOutpusStreamDecorator(stream)); + return decorateWithCompression(new NonClosingOutputStreamDecorator(stream)); } /** @@ -64,7 +64,7 @@ public final InputStream decorateWithCompression(InputStream stream) throws IOEx * @param stream the stream to decorate * @return an output stream that is decorated by the compression scheme. */ - protected abstract OutputStream decorateWithCompression(NonClosingOutpusStreamDecorator stream) + protected abstract OutputStream decorateWithCompression(NonClosingOutputStreamDecorator stream) throws IOException; /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/UncompressedStreamCompressionDecorator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/UncompressedStreamCompressionDecorator.java index 05ac250ea38bd..7fa053e069bf2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/UncompressedStreamCompressionDecorator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/UncompressedStreamCompressionDecorator.java @@ -20,7 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.runtime.util.NonClosingInputStreamDecorator; -import org.apache.flink.runtime.util.NonClosingOutpusStreamDecorator; +import org.apache.flink.runtime.util.NonClosingOutputStreamDecorator; import java.io.IOException; import java.io.InputStream; @@ -36,7 +36,7 @@ public class UncompressedStreamCompressionDecorator extends StreamCompressionDec private static final long serialVersionUID = 1L; @Override - protected OutputStream decorateWithCompression(NonClosingOutpusStreamDecorator stream) + protected OutputStream decorateWithCompression(NonClosingOutputStreamDecorator stream) throws IOException { return stream; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateHandleStreamImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateHandleStreamImpl.java index f9fd5042152c6..5a996b239bfe0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateHandleStreamImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateHandleStreamImpl.java @@ -100,7 +100,7 @@ public KeyGroupRange getKeyGroupRange() { @Nullable @Override public KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange) { - KeyGroupRange offsets = keyGroupRange.getIntersection(keyGroupRange); + KeyGroupRange offsets = this.keyGroupRange.getIntersection(keyGroupRange); if (offsets.getNumberOfKeyGroups() == 0) { return null; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/StateChangelogStorage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/StateChangelogStorage.java index 0bc26e3f05b2e..14794b40efa11 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/StateChangelogStorage.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/StateChangelogStorage.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.state.changelog; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.operators.MailboxExecutor; import org.apache.flink.runtime.io.AvailabilityProvider; import org.apache.flink.runtime.state.KeyGroupRange; @@ -30,7 +31,8 @@ @Internal public interface StateChangelogStorage extends AutoCloseable { - StateChangelogWriter createWriter(String operatorID, KeyGroupRange keyGroupRange); + StateChangelogWriter createWriter( + String operatorID, KeyGroupRange keyGroupRange, MailboxExecutor mailboxExecutor); StateChangelogHandleReader createReader(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/inmemory/InMemoryStateChangelogStorage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/inmemory/InMemoryStateChangelogStorage.java index c9bfdfbc5004a..6f49407411b5a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/inmemory/InMemoryStateChangelogStorage.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/inmemory/InMemoryStateChangelogStorage.java @@ -17,6 +17,7 @@ package org.apache.flink.runtime.state.changelog.inmemory; +import org.apache.flink.api.common.operators.MailboxExecutor; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.changelog.StateChangelogHandleReader; import org.apache.flink.runtime.state.changelog.StateChangelogStorage; @@ -28,7 +29,7 @@ public class InMemoryStateChangelogStorage @Override public InMemoryStateChangelogWriter createWriter( - String operatorID, KeyGroupRange keyGroupRange) { + String operatorID, KeyGroupRange keyGroupRange, MailboxExecutor mailboxExecutor) { return new InMemoryStateChangelogWriter(keyGroupRange); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/NonClosingOutpusStreamDecorator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/NonClosingOutputStreamDecorator.java similarity index 90% rename from flink-runtime/src/main/java/org/apache/flink/runtime/util/NonClosingOutpusStreamDecorator.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/util/NonClosingOutputStreamDecorator.java index dee7d7d005731..c40577375252a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/NonClosingOutpusStreamDecorator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/NonClosingOutputStreamDecorator.java @@ -25,9 +25,9 @@ /** Decorator for input streams that ignores calls to {@link OutputStream#close()}. */ @Internal -public class NonClosingOutpusStreamDecorator extends ForwardingOutputStream { +public class NonClosingOutputStreamDecorator extends ForwardingOutputStream { - public NonClosingOutpusStreamDecorator(OutputStream delegate) { + public NonClosingOutputStreamDecorator(OutputStream delegate) { super(delegate); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java index c62b0c3bde066..2de67d006d27e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java @@ -34,6 +34,7 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.jobmanager.DefaultJobGraphStore; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import org.apache.flink.runtime.jobmanager.JobGraphStore; @@ -569,6 +570,7 @@ public static JobGraphStore createJobGraphs( * @param configuration {@link Configuration} object * @param maxNumberOfCheckpointsToRetain The maximum number of checkpoints to retain * @param executor to run ZooKeeper callbacks + * @param restoreMode the mode in which the job is being restored * @return {@link DefaultCompletedCheckpointStore} instance * @throws Exception if the completed checkpoint store cannot be created */ @@ -578,7 +580,8 @@ public static CompletedCheckpointStore createCompletedCheckpoints( int maxNumberOfCheckpointsToRetain, SharedStateRegistryFactory sharedStateRegistryFactory, Executor ioExecutor, - Executor executor) + Executor executor, + RestoreMode restoreMode) throws Exception { checkNotNull(configuration, "Configuration"); @@ -597,7 +600,8 @@ public static CompletedCheckpointStore createCompletedCheckpoints( completedCheckpointStateHandleStore, ZooKeeperCheckpointStoreUtil.INSTANCE, completedCheckpoints, - sharedStateRegistryFactory.create(ioExecutor, completedCheckpoints), + sharedStateRegistryFactory.create( + ioExecutor, completedCheckpoints, restoreMode), executor); LOG.info( "Initialized {} in '{}' with {}.", diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java index c4388d410bf1b..0217ffdac9bc2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java @@ -391,7 +391,6 @@ List, String>> getAllAndLock( final String rootPath = "/"; boolean success = false; - retry: while (!success) { stateHandles.clear(); @@ -411,8 +410,13 @@ List, String>> getAllAndLock( final RetrievableStateHandle stateHandle = getAndLock(path); stateHandles.add(new Tuple2<>(stateHandle, path)); } catch (NotExistException ignored) { - // Concurrent deletion, retry - continue retry; + // The node is subject for deletion which can mean two things: + // 1. The state is marked for deletion: The cVersion of the node does not + // necessarily change. We're not interested in the state anymore, anyway. + // Therefore, this error can be ignored. + // 2. An actual concurrent deletion is going on. The child node is gone. + // That would affect the cVersion of the parent node and, as a consequence, + // would trigger a restart the logic through the while loop. } catch (IOException ioException) { LOG.warn( "Could not get all ZooKeeper children. Node {} contained " diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java index 15cc61e52de87..982df0c94c06f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.blob; import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; @@ -380,10 +379,7 @@ private void runBlobServerExpiresRecoveredTransientBlob(@Nullable JobID jobId) try (final BlobServer blobServer = createTestInstance(temporaryFolder.getAbsolutePath(), cleanupInterval)) { - CommonTestUtils.waitUntilCondition( - () -> !blob.exists(), - Deadline.fromNow(Duration.ofSeconds(cleanupInterval * 5L)), - "The transient blob has not been cleaned up automatically."); + CommonTestUtils.waitUntilCondition(() -> !blob.exists()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/PermanentBlobCacheTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/PermanentBlobCacheTest.java index d26156888b4a3..99599af225d56 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/PermanentBlobCacheTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/PermanentBlobCacheTest.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.blob; import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.testutils.CommonTestUtils; @@ -35,7 +34,6 @@ import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; -import java.time.Duration; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -107,10 +105,7 @@ public void permanentBlobCacheTimesOutRecoveredBlobs(@TempDir Path storageDirect try (final PermanentBlobCache permanentBlobCache = new PermanentBlobCache( configuration, storageDirectory.toFile(), new VoidBlobStore(), null)) { - CommonTestUtils.waitUntilCondition( - () -> !blobFile.exists(), - Deadline.fromNow(Duration.ofSeconds(cleanupInterval * 5L)), - "The permanent blob file was not cleaned up automatically."); + CommonTestUtils.waitUntilCondition(() -> !blobFile.exists()); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/TransientBlobCacheTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/TransientBlobCacheTest.java index db5d6407db9cc..d29b31b6dc628 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/TransientBlobCacheTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/TransientBlobCacheTest.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.blob; import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.testutils.CommonTestUtils; @@ -33,7 +32,6 @@ import java.io.File; import java.io.IOException; import java.nio.file.Path; -import java.time.Duration; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -92,9 +90,7 @@ public void transientBlobCacheTimesOutRecoveredBlobs(@TempDir Path storageDirect try (final TransientBlobCache transientBlobCache = new TransientBlobCache(configuration, storageDirectory.toFile(), null)) { - CommonTestUtils.waitUntilCondition( - () -> !blobFile.exists(), - Deadline.fromNow(Duration.ofSeconds(cleanupInterval * 5L))); + CommonTestUtils.waitUntilCondition(() -> !blobFile.exists()); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java index 36745153959c2..f242f92e38742 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; import org.apache.flink.runtime.persistence.PossibleInconsistentStateException; import org.apache.flink.runtime.state.InputChannelStateHandle; @@ -280,7 +281,7 @@ private static final class FailingCompletedCheckpointStore public FailingCompletedCheckpointStore(Exception addCheckpointFailure) { super( SharedStateRegistry.DEFAULT_FACTORY.create( - Executors.directExecutor(), emptyList())); + Executors.directExecutor(), emptyList(), RestoreMode.DEFAULT)); this.addCheckpointFailure = addCheckpointFailure; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java index 3b839739663fc..3e9fb12d51a71 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration.CheckpointCoordinatorConfigurationBuilder; import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; @@ -234,7 +235,7 @@ private void testRestoreLatestCheckpointedState( final ExecutionGraph executionGraph = createExecutionGraph(vertices); final EmbeddedCompletedCheckpointStore store = new EmbeddedCompletedCheckpointStore( - completedCheckpoints.size(), completedCheckpoints); + completedCheckpoints.size(), completedCheckpoints, RestoreMode.DEFAULT); // set up the coordinator and validate the initial state final CheckpointCoordinator coordinator = @@ -778,7 +779,8 @@ public void testStateRecoveryWithTopologyChange(TestScaleType scaleType) throws // set up the coordinator and validate the initial state SharedStateRegistry sharedStateRegistry = - SharedStateRegistry.DEFAULT_FACTORY.create(Executors.directExecutor(), emptyList()); + SharedStateRegistry.DEFAULT_FACTORY.create( + Executors.directExecutor(), emptyList(), RestoreMode.DEFAULT); CheckpointCoordinator coord = new CheckpointCoordinatorBuilder() .setExecutionGraph(newGraph) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java index 30af82fbe7d59..9d3a87ddd848f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java @@ -38,6 +38,7 @@ import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; import org.apache.flink.runtime.jobmaster.LogicalSlot; import org.apache.flink.runtime.jobmaster.TestingLogicalSlotBuilder; @@ -2867,163 +2868,178 @@ public void testCheckpointStatsTrackerRestoreCallback() throws Exception { @Test public void testSharedStateRegistrationOnRestore() throws Exception { - JobVertexID jobVertexID1 = new JobVertexID(); + for (RestoreMode restoreMode : RestoreMode.values()) { + JobVertexID jobVertexID1 = new JobVertexID(); - int parallelism1 = 2; - int maxParallelism1 = 4; + int parallelism1 = 2; + int maxParallelism1 = 4; - ExecutionGraph graph = - new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() - .addJobVertex(jobVertexID1, parallelism1, maxParallelism1) - .build(); + ExecutionGraph graph = + new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() + .addJobVertex(jobVertexID1, parallelism1, maxParallelism1) + .build(); - ExecutionJobVertex jobVertex1 = graph.getJobVertex(jobVertexID1); + ExecutionJobVertex jobVertex1 = graph.getJobVertex(jobVertexID1); - List checkpoints = Collections.emptyList(); - SharedStateRegistry firstInstance = - SharedStateRegistry.DEFAULT_FACTORY.create( - org.apache.flink.util.concurrent.Executors.directExecutor(), checkpoints); - final EmbeddedCompletedCheckpointStore store = - new EmbeddedCompletedCheckpointStore(10, checkpoints, firstInstance); + List checkpoints = Collections.emptyList(); + SharedStateRegistry firstInstance = + SharedStateRegistry.DEFAULT_FACTORY.create( + org.apache.flink.util.concurrent.Executors.directExecutor(), + checkpoints, + restoreMode); + final EmbeddedCompletedCheckpointStore store = + new EmbeddedCompletedCheckpointStore(10, checkpoints, firstInstance); - // set up the coordinator and validate the initial state - final CheckpointCoordinatorBuilder coordinatorBuilder = - new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) - .setTimer(manuallyTriggeredScheduledExecutor); - final CheckpointCoordinator coordinator = - coordinatorBuilder.setCompletedCheckpointStore(store).build(); + // set up the coordinator and validate the initial state + final CheckpointCoordinatorBuilder coordinatorBuilder = + new CheckpointCoordinatorBuilder() + .setExecutionGraph(graph) + .setTimer(manuallyTriggeredScheduledExecutor); + final CheckpointCoordinator coordinator = + coordinatorBuilder.setCompletedCheckpointStore(store).build(); - final int numCheckpoints = 3; + final int numCheckpoints = 3; - List keyGroupPartitions1 = - StateAssignmentOperation.createKeyGroupPartitions(maxParallelism1, parallelism1); + List keyGroupPartitions1 = + StateAssignmentOperation.createKeyGroupPartitions( + maxParallelism1, parallelism1); - for (int i = 0; i < numCheckpoints; ++i) { - performIncrementalCheckpoint( - graph.getJobID(), coordinator, jobVertex1, keyGroupPartitions1, i); - } + for (int i = 0; i < numCheckpoints; ++i) { + performIncrementalCheckpoint( + graph.getJobID(), coordinator, jobVertex1, keyGroupPartitions1, i); + } - List completedCheckpoints = coordinator.getSuccessfulCheckpoints(); - assertEquals(numCheckpoints, completedCheckpoints.size()); + List completedCheckpoints = coordinator.getSuccessfulCheckpoints(); + assertEquals(numCheckpoints, completedCheckpoints.size()); - int sharedHandleCount = 0; + int sharedHandleCount = 0; - List> sharedHandlesByCheckpoint = - new ArrayList<>(numCheckpoints); + List> sharedHandlesByCheckpoint = + new ArrayList<>(numCheckpoints); - for (int i = 0; i < numCheckpoints; ++i) { - sharedHandlesByCheckpoint.add(new HashMap<>(2)); - } + for (int i = 0; i < numCheckpoints; ++i) { + sharedHandlesByCheckpoint.add(new HashMap<>(2)); + } - int cp = 0; - for (CompletedCheckpoint completedCheckpoint : completedCheckpoints) { - for (OperatorState taskState : completedCheckpoint.getOperatorStates().values()) { - for (OperatorSubtaskState subtaskState : taskState.getStates()) { - for (KeyedStateHandle keyedStateHandle : subtaskState.getManagedKeyedState()) { - // test we are once registered with the current registry - verify(keyedStateHandle, times(1)) - .registerSharedStates( - firstInstance, completedCheckpoint.getCheckpointID()); - IncrementalRemoteKeyedStateHandle incrementalKeyedStateHandle = - (IncrementalRemoteKeyedStateHandle) keyedStateHandle; - - sharedHandlesByCheckpoint - .get(cp) - .putAll(incrementalKeyedStateHandle.getSharedState()); - - for (StreamStateHandle streamStateHandle : - incrementalKeyedStateHandle.getSharedState().values()) { - assertTrue( - !(streamStateHandle instanceof PlaceholderStreamStateHandle)); - verify(streamStateHandle, never()).discardState(); - ++sharedHandleCount; - } + int cp = 0; + for (CompletedCheckpoint completedCheckpoint : completedCheckpoints) { + for (OperatorState taskState : completedCheckpoint.getOperatorStates().values()) { + for (OperatorSubtaskState subtaskState : taskState.getStates()) { + for (KeyedStateHandle keyedStateHandle : + subtaskState.getManagedKeyedState()) { + // test we are once registered with the current registry + verify(keyedStateHandle, times(1)) + .registerSharedStates( + firstInstance, completedCheckpoint.getCheckpointID()); + IncrementalRemoteKeyedStateHandle incrementalKeyedStateHandle = + (IncrementalRemoteKeyedStateHandle) keyedStateHandle; + + sharedHandlesByCheckpoint + .get(cp) + .putAll(incrementalKeyedStateHandle.getSharedState()); + + for (StreamStateHandle streamStateHandle : + incrementalKeyedStateHandle.getSharedState().values()) { + assertTrue( + !(streamStateHandle + instanceof PlaceholderStreamStateHandle)); + verify(streamStateHandle, never()).discardState(); + ++sharedHandleCount; + } + + for (StreamStateHandle streamStateHandle : + incrementalKeyedStateHandle.getPrivateState().values()) { + verify(streamStateHandle, never()).discardState(); + } - for (StreamStateHandle streamStateHandle : - incrementalKeyedStateHandle.getPrivateState().values()) { - verify(streamStateHandle, never()).discardState(); + verify(incrementalKeyedStateHandle.getMetaStateHandle(), never()) + .discardState(); } - verify(incrementalKeyedStateHandle.getMetaStateHandle(), never()) - .discardState(); + verify(subtaskState, never()).discardState(); } - - verify(subtaskState, never()).discardState(); } + ++cp; } - ++cp; - } - // 2 (parallelism) x (1 (CP0) + 2 (CP1) + 2 (CP2)) = 10 - assertEquals(10, sharedHandleCount); + // 2 (parallelism) x (1 (CP0) + 2 (CP1) + 2 (CP2)) = 10 + assertEquals(10, sharedHandleCount); - // discard CP0 - store.removeOldestCheckpoint(); + // discard CP0 + store.removeOldestCheckpoint(); - // we expect no shared state was discarded because the state of CP0 is still referenced by - // CP1 - for (Map cpList : sharedHandlesByCheckpoint) { - for (StreamStateHandle streamStateHandle : cpList.values()) { - verify(streamStateHandle, never()).discardState(); + // we expect no shared state was discarded because the state of CP0 is still referenced + // by + // CP1 + for (Map cpList : sharedHandlesByCheckpoint) { + for (StreamStateHandle streamStateHandle : cpList.values()) { + verify(streamStateHandle, never()).discardState(); + } } - } - // shutdown the store - store.shutdown(JobStatus.SUSPENDED, new CheckpointsCleaner()); - - // restore the store - Set tasks = new HashSet<>(); - tasks.add(jobVertex1); - - assertEquals(JobStatus.SUSPENDED, store.getShutdownStatus().orElse(null)); - SharedStateRegistry secondInstance = - SharedStateRegistry.DEFAULT_FACTORY.create( - org.apache.flink.util.concurrent.Executors.directExecutor(), - store.getAllCheckpoints()); - final EmbeddedCompletedCheckpointStore secondStore = - new EmbeddedCompletedCheckpointStore(10, store.getAllCheckpoints(), secondInstance); - final CheckpointCoordinator secondCoordinator = - coordinatorBuilder.setCompletedCheckpointStore(secondStore).build(); - assertTrue(secondCoordinator.restoreLatestCheckpointedStateToAll(tasks, false)); - - // validate that all shared states are registered again after the recovery. - cp = 0; - for (CompletedCheckpoint completedCheckpoint : completedCheckpoints) { - for (OperatorState taskState : completedCheckpoint.getOperatorStates().values()) { - for (OperatorSubtaskState subtaskState : taskState.getStates()) { - for (KeyedStateHandle keyedStateHandle : subtaskState.getManagedKeyedState()) { - VerificationMode verificationMode; - // test we are once registered with the new registry - if (cp > 0) { - verificationMode = times(1); - } else { - verificationMode = never(); - } + // shutdown the store + store.shutdown(JobStatus.SUSPENDED, new CheckpointsCleaner()); + + // restore the store + Set tasks = new HashSet<>(); + tasks.add(jobVertex1); + + assertEquals(JobStatus.SUSPENDED, store.getShutdownStatus().orElse(null)); + SharedStateRegistry secondInstance = + SharedStateRegistry.DEFAULT_FACTORY.create( + org.apache.flink.util.concurrent.Executors.directExecutor(), + store.getAllCheckpoints(), + restoreMode); + final EmbeddedCompletedCheckpointStore secondStore = + new EmbeddedCompletedCheckpointStore( + 10, store.getAllCheckpoints(), secondInstance); + final CheckpointCoordinator secondCoordinator = + coordinatorBuilder.setCompletedCheckpointStore(secondStore).build(); + assertTrue(secondCoordinator.restoreLatestCheckpointedStateToAll(tasks, false)); + + // validate that all shared states are registered again after the recovery. + cp = 0; + for (CompletedCheckpoint completedCheckpoint : completedCheckpoints) { + for (OperatorState taskState : completedCheckpoint.getOperatorStates().values()) { + for (OperatorSubtaskState subtaskState : taskState.getStates()) { + for (KeyedStateHandle keyedStateHandle : + subtaskState.getManagedKeyedState()) { + VerificationMode verificationMode; + // test we are once registered with the new registry + if (cp > 0) { + verificationMode = times(1); + } else { + verificationMode = never(); + } - // check that all are registered with the new registry - verify(keyedStateHandle, verificationMode) - .registerSharedStates( - secondInstance, completedCheckpoint.getCheckpointID()); + // check that all are registered with the new registry + verify(keyedStateHandle, verificationMode) + .registerSharedStates( + secondInstance, completedCheckpoint.getCheckpointID()); + } } } + ++cp; } - ++cp; - } - // discard CP1 - secondStore.removeOldestCheckpoint(); + // discard CP1 + secondStore.removeOldestCheckpoint(); - // we expect that all shared state from CP0 is no longer referenced and discarded. CP2 is - // still live and also - // references the state from CP1, so we expect they are not discarded. - verifyDiscard(sharedHandlesByCheckpoint, cpId -> cpId == 0 ? times(1) : never()); + // we expect that all shared state from CP0 is no longer referenced and discarded. CP2 + // is + // still live and also + // references the state from CP1, so we expect they are not discarded. + verifyDiscard( + sharedHandlesByCheckpoint, + cpId -> restoreMode == RestoreMode.CLAIM && cpId == 0 ? times(1) : never()); - // discard CP2 - secondStore.removeOldestCheckpoint(); + // discard CP2 + secondStore.removeOldestCheckpoint(); - // still expect shared state not to be discarded because it may be used in later checkpoints - verifyDiscard(sharedHandlesByCheckpoint, cpId -> cpId == 1 ? never() : atLeast(0)); + // still expect shared state not to be discarded because it may be used in later + // checkpoints + verifyDiscard(sharedHandlesByCheckpoint, cpId -> cpId == 1 ? never() : atLeast(0)); + } } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java index ddf446bf66501..643c09d95c684 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java @@ -35,6 +35,7 @@ import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.ExecutorUtils; import org.apache.flink.util.TestLogger; +import org.apache.flink.util.concurrent.FutureUtils; import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor; import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter; @@ -835,7 +836,9 @@ public UnstableCheckpointIDCounter(Predicate checkpointFailurePredicate) { public void start() {} @Override - public void shutdown(JobStatus jobStatus) throws Exception {} + public CompletableFuture shutdown(JobStatus jobStatus) { + return FutureUtils.completedVoidFuture(); + } @Override public long getAndIncrement() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounterTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounterTestBase.java index 1dcfd3344bcd4..59933e856b200 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounterTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounterTestBase.java @@ -58,7 +58,7 @@ public void testCounterIsNeverNegative() throws Exception { counter.start(); assertThat(counter.get()).isGreaterThanOrEqualTo(0L); } finally { - counter.shutdown(JobStatus.FINISHED); + counter.shutdown(JobStatus.FINISHED).join(); } } @@ -78,7 +78,7 @@ public void testSerialIncrementAndGet() throws Exception { assertThat(counter.get()).isEqualTo(4); assertThat(counter.getAndIncrement()).isEqualTo(4); } finally { - counter.shutdown(JobStatus.FINISHED); + counter.shutdown(JobStatus.FINISHED).join(); } } @@ -136,7 +136,7 @@ public void testConcurrentGetAndIncrement() throws Exception { executor.shutdown(); } - counter.shutdown(JobStatus.FINISHED); + counter.shutdown(JobStatus.FINISHED).join(); } } @@ -161,7 +161,7 @@ public void testSetCount() throws Exception { assertThat(counter.get()).isEqualTo(1338); assertThat(counter.getAndIncrement()).isEqualTo(1338); - counter.shutdown(JobStatus.FINISHED); + counter.shutdown(JobStatus.FINISHED).join(); } /** Task repeatedly incrementing the {@link CheckpointIDCounter}. */ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointTest.java index b6461a9c98462..a0b67cd0dfa29 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointTest.java @@ -23,6 +23,7 @@ import org.apache.flink.core.testutils.CommonTestUtils; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.SharedStateRegistryImpl; import org.apache.flink.runtime.state.testutils.EmptyStreamStateHandle; @@ -235,7 +236,7 @@ public void testRegisterStatesAtRegistry() { null); SharedStateRegistry sharedStateRegistry = new SharedStateRegistryImpl(); - checkpoint.registerSharedStatesAfterRestored(sharedStateRegistry); + checkpoint.registerSharedStatesAfterRestored(sharedStateRegistry, RestoreMode.DEFAULT); verify(state, times(1)).registerSharedStates(sharedStateRegistry, 0L); } @@ -267,7 +268,7 @@ public void testCleanUpOnSubsume() throws Exception { null); SharedStateRegistry sharedStateRegistry = new SharedStateRegistryImpl(); - checkpoint.registerSharedStatesAfterRestored(sharedStateRegistry); + checkpoint.registerSharedStatesAfterRestored(sharedStateRegistry, RestoreMode.DEFAULT); verify(state, times(1)).registerSharedStates(sharedStateRegistry, 0L); // Subsume diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreTest.java index 064e26a9b9fb6..22cd5dd021cd4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreTest.java @@ -23,6 +23,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.execution.SavepointFormatType; import org.apache.flink.core.testutils.FlinkMatchers; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.persistence.StateHandleStore; import org.apache.flink.runtime.persistence.TestingRetrievableStateStorageHelper; import org.apache.flink.runtime.persistence.TestingStateHandleStore; @@ -397,7 +398,9 @@ public long nameToCheckpointID(String name) { DefaultCompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( stateHandleStore, checkpointStoreUtil), SharedStateRegistry.DEFAULT_FACTORY.create( - org.apache.flink.util.concurrent.Executors.directExecutor(), emptyList()), + org.apache.flink.util.concurrent.Executors.directExecutor(), + emptyList(), + RestoreMode.DEFAULT), executorService); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryTest.java index 58a0d954318dd..7203edfc7099a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryTest.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.Executors; @@ -49,7 +50,8 @@ public void testFactoryWithoutCheckpointStoreRecovery() throws Exception { firstJobId, 1, SharedStateRegistry.DEFAULT_FACTORY, - Executors.directExecutor())); + Executors.directExecutor(), + RestoreMode.DEFAULT)); assertThrows( UnsupportedOperationException.class, () -> @@ -57,7 +59,8 @@ public void testFactoryWithoutCheckpointStoreRecovery() throws Exception { firstJobId, 1, SharedStateRegistry.DEFAULT_FACTORY, - Executors.directExecutor())); + Executors.directExecutor(), + RestoreMode.DEFAULT)); final JobID secondJobId = new JobID(); assertSame( @@ -66,7 +69,8 @@ public void testFactoryWithoutCheckpointStoreRecovery() throws Exception { secondJobId, 1, SharedStateRegistry.DEFAULT_FACTORY, - Executors.directExecutor())); + Executors.directExecutor(), + RestoreMode.DEFAULT)); assertThrows( UnsupportedOperationException.class, () -> @@ -74,6 +78,7 @@ public void testFactoryWithoutCheckpointStoreRecovery() throws Exception { secondJobId, 1, SharedStateRegistry.DEFAULT_FACTORY, - Executors.directExecutor())); + Executors.directExecutor(), + RestoreMode.DEFAULT)); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointIDCounter.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointIDCounter.java index 2365ed59789cf..466a69a25d94f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointIDCounter.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointIDCounter.java @@ -18,16 +18,18 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobStatus; +import org.apache.flink.util.concurrent.FutureUtils; import java.util.concurrent.CompletableFuture; import java.util.function.Consumer; +import java.util.function.Function; import java.util.function.Supplier; /** Test {@link CheckpointIDCounter} implementation for testing the shutdown behavior. */ public final class TestingCheckpointIDCounter implements CheckpointIDCounter { private final Runnable startRunnable; - private final Consumer shutdownConsumer; + private final Function> shutdownFunction; private final Supplier getAndIncrementSupplier; private final Supplier getSupplier; private final Consumer setCountConsumer; @@ -36,18 +38,22 @@ public static TestingCheckpointIDCounter createStoreWithShutdownCheckAndNoStartA CompletableFuture shutdownFuture) { return TestingCheckpointIDCounter.builder() .withStartRunnable(() -> {}) - .withShutdownConsumer(shutdownFuture::complete) + .withShutdownConsumer( + jobStatus -> { + shutdownFuture.complete(jobStatus); + return FutureUtils.completedVoidFuture(); + }) .build(); } private TestingCheckpointIDCounter( Runnable startRunnable, - Consumer shutdownConsumer, + Function> shutdownFunction, Supplier getAndIncrementSupplier, Supplier getSupplier, Consumer setCountConsumer) { this.startRunnable = startRunnable; - this.shutdownConsumer = shutdownConsumer; + this.shutdownFunction = shutdownFunction; this.getAndIncrementSupplier = getAndIncrementSupplier; this.getSupplier = getSupplier; this.setCountConsumer = setCountConsumer; @@ -59,8 +65,8 @@ public void start() { } @Override - public void shutdown(JobStatus jobStatus) { - shutdownConsumer.accept(jobStatus); + public CompletableFuture shutdown(JobStatus jobStatus) { + return shutdownFunction.apply(jobStatus); } @Override @@ -86,7 +92,7 @@ public static Builder builder() { public static class Builder { private Runnable startRunnable; - private Consumer shutdownConsumer; + private Function> shutdownFunction; private Supplier getAndIncrementSupplier; private Supplier getSupplier; private Consumer setCountConsumer; @@ -96,8 +102,9 @@ public Builder withStartRunnable(Runnable startRunnable) { return this; } - public Builder withShutdownConsumer(Consumer shutdownConsumer) { - this.shutdownConsumer = shutdownConsumer; + public Builder withShutdownConsumer( + Function> shutdownFunction) { + this.shutdownFunction = shutdownFunction; return this; } @@ -119,7 +126,7 @@ public Builder withSetCountConsumer(Consumer setCountConsumer) { public TestingCheckpointIDCounter build() { return new TestingCheckpointIDCounter( startRunnable, - shutdownConsumer, + shutdownFunction, getAndIncrementSupplier, getSupplier, setCountConsumer); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointRecoveryFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointRecoveryFactory.java index e164543d0ab81..687196dff827c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointRecoveryFactory.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointRecoveryFactory.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.state.SharedStateRegistryFactory; import java.util.concurrent.Executor; @@ -39,7 +40,8 @@ public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, SharedStateRegistryFactory sharedStateRegistryFactory, - Executor ioExecutor) { + Executor ioExecutor, + RestoreMode restoreMode) { return store; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZKCheckpointIDCounterMultiServersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZKCheckpointIDCounterMultiServersTest.java index 5f303e1bcc897..c18408f08d575 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZKCheckpointIDCounterMultiServersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZKCheckpointIDCounterMultiServersTest.java @@ -33,9 +33,7 @@ import org.junit.Rule; import org.junit.Test; -import java.util.concurrent.atomic.AtomicLong; - -import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.number.OrderingComparison.greaterThan; import static org.junit.Assert.assertThat; /** Tests for {@link ZooKeeperCheckpointIDCounter} in a ZooKeeper ensemble. */ @@ -71,22 +69,14 @@ public void testRecoveredAfterConnectionLoss() throws Exception { curatorFrameworkWrapper.asCuratorFramework(), listener); idCounter.start(); - AtomicLong localCounter = new AtomicLong(1L); - - assertThat( - "ZooKeeperCheckpointIDCounter doesn't properly work.", - idCounter.getAndIncrement(), - is(localCounter.getAndIncrement())); + final long initialID = idCounter.getAndIncrement(); zooKeeperResource.restart(); connectionLossLatch.await(); reconnectedLatch.await(); - assertThat( - "ZooKeeperCheckpointIDCounter doesn't properly work after reconnected.", - idCounter.getAndIncrement(), - is(localCounter.getAndIncrement())); + assertThat(idCounter.getAndIncrement(), greaterThan(initialID)); } finally { curatorFrameworkWrapper.close(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounterITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounterITCase.java index 798ea795589fe..4e9c0d4e55470 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounterITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounterITCase.java @@ -19,16 +19,21 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobStatus; +import org.apache.flink.core.testutils.FlinkAssertions; +import org.apache.flink.runtime.util.ZooKeeperUtils; import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment; import org.apache.flink.shaded.curator5.org.apache.curator.framework.CuratorFramework; +import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import java.util.concurrent.ExecutionException; + import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** * Unit tests for the {@link ZooKeeperCheckpointIDCounter}. The tests are inherited from the test @@ -38,19 +43,21 @@ class ZooKeeperCheckpointIDCounterITCase extends CheckpointIDCounterTestBase { private static ZooKeeperTestEnvironment zookeeper; - @BeforeAll - public static void setUp() throws Exception { + @BeforeEach + void setup() { zookeeper = new ZooKeeperTestEnvironment(1); } - @AfterAll - private static void tearDown() throws Exception { - zookeeper.shutdown(); + @AfterEach + void tearDown() throws Exception { + cleanAndStopZooKeeperIfRunning(); } - @BeforeEach - private void cleanUp() throws Exception { - zookeeper.deleteAll(); + private void cleanAndStopZooKeeperIfRunning() throws Exception { + if (zookeeper.getClient().isStarted()) { + zookeeper.deleteAll(); + zookeeper.shutdown(); + } } /** Tests that counter node is removed from ZooKeeper after shutdown. */ @@ -62,10 +69,73 @@ public void testShutdownRemovesState() throws Exception { CuratorFramework client = zookeeper.getClient(); assertThat(client.checkExists().forPath(counter.getPath())).isNotNull(); - counter.shutdown(JobStatus.FINISHED); + counter.shutdown(JobStatus.FINISHED).join(); + assertThat(client.checkExists().forPath(counter.getPath())).isNull(); + } + + @Test + public void testIdempotentShutdown() throws Exception { + ZooKeeperCheckpointIDCounter counter = createCheckpointIdCounter(); + counter.start(); + + CuratorFramework client = zookeeper.getClient(); + counter.shutdown(JobStatus.FINISHED).join(); + + // shutdown shouldn't fail due to missing path + counter.shutdown(JobStatus.FINISHED).join(); assertThat(client.checkExists().forPath(counter.getPath())).isNull(); } + @Test + public void testShutdownWithFailureDueToMissingConnection() throws Exception { + ZooKeeperCheckpointIDCounter counter = createCheckpointIdCounter(); + counter.start(); + + cleanAndStopZooKeeperIfRunning(); + + assertThatThrownBy(() -> counter.shutdown(JobStatus.FINISHED).get()) + .as("The shutdown should fail because of the client connection being dropped.") + .isInstanceOf(ExecutionException.class) + .hasCauseExactlyInstanceOf(IllegalStateException.class); + } + + @Test + public void testShutdownWithFailureDueToExistingChildNodes() throws Exception { + final ZooKeeperCheckpointIDCounter counter = createCheckpointIdCounter(); + counter.start(); + + final CuratorFramework client = + ZooKeeperUtils.useNamespaceAndEnsurePath(zookeeper.getClient(), "/"); + final String counterNodePath = ZooKeeperUtils.generateZookeeperPath(counter.getPath()); + final String childNodePath = + ZooKeeperUtils.generateZookeeperPath( + counterNodePath, "unexpected-child-node-causing-a-failure"); + client.create().forPath(childNodePath); + + final String namespacedCounterNodePath = + ZooKeeperUtils.generateZookeeperPath(client.getNamespace(), counterNodePath); + final Throwable expectedRootCause = + KeeperException.create(KeeperException.Code.NOTEMPTY, namespacedCounterNodePath); + assertThatThrownBy(() -> counter.shutdown(JobStatus.FINISHED).get()) + .as( + "The shutdown should fail because of a child node being present and the shutdown not performing an explicit recursive deletion.") + .isInstanceOf(ExecutionException.class) + .extracting(FlinkAssertions::chainOfCauses, FlinkAssertions.STREAM_THROWABLE) + .anySatisfy( + cause -> + assertThat(cause) + .isInstanceOf(expectedRootCause.getClass()) + .hasMessage(expectedRootCause.getMessage())); + + client.delete().forPath(childNodePath); + counter.shutdown(JobStatus.FINISHED).join(); + + assertThat(client.checkExists().forPath(counterNodePath)) + .as( + "A retry of the shutdown should have worked now after the root cause was resolved.") + .isNull(); + } + /** Tests that counter node is NOT removed from ZooKeeper after suspend. */ @Test public void testSuspendKeepsState() throws Exception { @@ -75,13 +145,14 @@ public void testSuspendKeepsState() throws Exception { CuratorFramework client = zookeeper.getClient(); assertThat(client.checkExists().forPath(counter.getPath())).isNotNull(); - counter.shutdown(JobStatus.SUSPENDED); + counter.shutdown(JobStatus.SUSPENDED).join(); assertThat(client.checkExists().forPath(counter.getPath())).isNotNull(); } @Override protected ZooKeeperCheckpointIDCounter createCheckpointIdCounter() throws Exception { return new ZooKeeperCheckpointIDCounter( - zookeeper.getClient(), new DefaultLastStateConnectionStateListener()); + ZooKeeperUtils.useNamespaceAndEnsurePath(zookeeper.getClient(), "/"), + new DefaultLastStateConnectionStateListener()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java index 240955927bbce..d3d705d51f719 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java @@ -20,7 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; -import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.state.RetrievableStateHandle; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.SharedStateRegistryImpl; @@ -41,7 +41,6 @@ import org.junit.Test; import java.io.Serializable; -import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -92,7 +91,8 @@ protected CompletedCheckpointStore createRecoveredCompletedCheckpointStore( checkpointStoreUtil, DefaultCompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( checkpointsInZooKeeper, checkpointStoreUtil), - SharedStateRegistry.DEFAULT_FACTORY.create(Executors.directExecutor(), emptyList()), + SharedStateRegistry.DEFAULT_FACTORY.create( + Executors.directExecutor(), emptyList(), RestoreMode.DEFAULT), executor); } @@ -383,8 +383,7 @@ public void testChekpointingPausesAndResumeWhenTooManyCheckpoints() throws Excep CommonTestUtils.waitUntilCondition( () -> checkpointsCleaner.getNumberOfCheckpointsToClean() - == nbCheckpointsSubmittedForCleaning, - Deadline.fromNow(Duration.ofSeconds(3))); + == nbCheckpointsSubmittedForCleaning); assertEquals( nbCheckpointsSubmittedForCleaning, checkpointsCleaner.getNumberOfCheckpointsToClean()); @@ -401,8 +400,7 @@ public void testChekpointingPausesAndResumeWhenTooManyCheckpoints() throws Excep CommonTestUtils.waitUntilCondition( () -> checkpointsCleaner.getNumberOfCheckpointsToClean() - < nbCheckpointsSubmittedForCleaning, - Deadline.fromNow(Duration.ofSeconds(3))); + < nbCheckpointsSubmittedForCleaning); // some checkpoints were cleaned assertTrue( checkpointsCleaner.getNumberOfCheckpointsToClean() diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java index 804d285ae28dd..cd8561157cb70 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java @@ -25,6 +25,7 @@ import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.core.testutils.FlinkMatchers; import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.operators.testutils.ExpectedTestException; import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler; import org.apache.flink.runtime.state.RetrievableStateHandle; @@ -196,7 +197,8 @@ private CompletedCheckpointStore createZooKeeperCheckpointStore(CuratorFramework checkpointsInZooKeeper, zooKeeperCheckpointStoreUtil, Collections.emptyList(), - SharedStateRegistry.DEFAULT_FACTORY.create(Executors.directExecutor(), emptyList()), + SharedStateRegistry.DEFAULT_FACTORY.create( + Executors.directExecutor(), emptyList(), RestoreMode.DEFAULT), Executors.directExecutor()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java index 22d32d509146b..77cb28f169f64 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java @@ -19,7 +19,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobServer; @@ -36,7 +35,6 @@ import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.util.TestingFatalErrorHandlerResource; import org.apache.flink.util.TestLogger; -import org.apache.flink.util.TimeUtils; import org.junit.After; import org.junit.AfterClass; @@ -69,8 +67,7 @@ public static void teardownClass() throws Exception { static void awaitStatus(DispatcherGateway dispatcherGateway, JobID jobId, JobStatus status) throws Exception { CommonTestUtils.waitUntilCondition( - () -> status.equals(dispatcherGateway.requestJobStatus(jobId, TIMEOUT).get()), - Deadline.fromNow(TimeUtils.toDuration(TIMEOUT))); + () -> status.equals(dispatcherGateway.requestJobStatus(jobId, TIMEOUT).get())); } @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -106,7 +103,6 @@ public void setUp() throws Exception { protected TestingDispatcher.Builder createTestingDispatcherBuilder() { return TestingDispatcher.builder() - .setRpcService(rpcService) .setConfiguration(configuration) .setHeartbeatServices(heartbeatServices) .setHighAvailabilityServices(haServices) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java index 8cfb1ad017532..2383eae943c80 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java @@ -19,7 +19,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; -import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.configuration.CleanupOptions; import org.apache.flink.core.testutils.FlinkMatchers; import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.checkpoint.EmbeddedCompletedCheckpointStore; @@ -34,6 +34,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobGraphBuilder; import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; import org.apache.flink.runtime.jobmanager.JobGraphStore; @@ -49,8 +50,6 @@ import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.TestingJobGraphStore; import org.apache.flink.runtime.testutils.TestingJobResultStore; -import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.TimeUtils; import org.apache.flink.util.concurrent.FutureUtils; import org.hamcrest.CoreMatchers; @@ -60,14 +59,11 @@ import org.junit.Before; import org.junit.Test; -import java.time.Duration; import java.util.Collections; import java.util.List; -import java.util.Optional; import java.util.UUID; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ForkJoinPool; import java.util.concurrent.LinkedBlockingQueue; @@ -89,7 +85,11 @@ public void setUp() throws Exception { super.setUp(); haServices.setCheckpointRecoveryFactory( new PerJobCheckpointRecoveryFactory( - (maxCheckpoints, previous, sharedStateRegistryFactory, ioExecutor) -> { + (maxCheckpoints, + previous, + sharedStateRegistryFactory, + ioExecutor, + restoreMode) -> { if (previous != null) { // First job cleanup still succeeded for the // CompletedCheckpointStore because the JobGraph cleanup happens @@ -100,13 +100,17 @@ public void setUp() throws Exception { maxCheckpoints, previous.getAllCheckpoints(), sharedStateRegistryFactory.create( - ioExecutor, previous.getAllCheckpoints())); + ioExecutor, + previous.getAllCheckpoints(), + restoreMode)); } return new EmbeddedCompletedCheckpointStore( maxCheckpoints, Collections.emptyList(), sharedStateRegistryFactory.create( - ioExecutor, Collections.emptyList())); + ioExecutor, + Collections.emptyList(), + RestoreMode.DEFAULT)); })); } @@ -133,12 +137,23 @@ public void testCleanupThroughRetries() throws Exception { final int numberOfErrors = 5; final RuntimeException temporaryError = new RuntimeException("Expected RuntimeException: Unable to remove job graph."); + final AtomicInteger failureCount = new AtomicInteger(numberOfErrors); final JobGraphStore jobGraphStore = - createAndStartJobGraphStoreWithCleanupFailures( - numberOfErrors, - temporaryError, - actualGlobalCleanupCallCount, - successfulCleanupLatch); + TestingJobGraphStore.newBuilder() + .setGlobalCleanupFunction( + (ignoredJobId, ignoredExecutor) -> { + actualGlobalCleanupCallCount.incrementAndGet(); + + if (failureCount.getAndDecrement() > 0) { + return FutureUtils.completedExceptionally(temporaryError); + } + + successfulCleanupLatch.trigger(); + return FutureUtils.completedVoidFuture(); + }) + .build(); + + jobGraphStore.start(NoOpJobGraphListener.INSTANCE); haServices.setJobGraphStore(jobGraphStore); // Construct leader election service. @@ -162,7 +177,7 @@ public void testCleanupThroughRetries() throws Exception { haServices, UnregisteredMetricGroups .createUnregisteredJobManagerMetricGroup())) - .build(); + .build(rpcService); dispatcher.start(); toTerminate.add(dispatcher); @@ -183,9 +198,7 @@ public void testCleanupThroughRetries() throws Exception { IsEmptyCollection.empty()); CommonTestUtils.waitUntilCondition( - () -> haServices.getJobResultStore().hasJobResultEntry(jobId), - Deadline.fromNow(Duration.ofMinutes(5)), - "The JobResultStore should have this job marked as clean."); + () -> haServices.getJobResultStore().hasJobResultEntry(jobId)); } @Test @@ -210,15 +223,12 @@ public void testCleanupNotCancellable() throws Exception { final Dispatcher dispatcher = createTestingDispatcherBuilder() .setJobManagerRunnerRegistry(jobManagerRunnerRegistry) - .build(); + .build(rpcService); dispatcher.start(); toTerminate.add(dispatcher); - CommonTestUtils.waitUntilCondition( - () -> jobManagerRunnerEntry.get() != null, - Deadline.fromNow(Duration.ofSeconds(10)), - "JobManagerRunner wasn't loaded in time."); + CommonTestUtils.waitUntilCondition(() -> jobManagerRunnerEntry.get() != null); assertThat( "The JobResultStore should have this job still marked as dirty.", @@ -237,9 +247,7 @@ public void testCleanupNotCancellable() throws Exception { jobManagerRunnerCleanupFuture.complete(null); CommonTestUtils.waitUntilCondition( - () -> haServices.getJobResultStore().hasCleanJobResultEntry(jobId), - Deadline.fromNow(Duration.ofSeconds(60)), - "The JobResultStore should have this job marked as clean now."); + () -> haServices.getJobResultStore().hasCleanJobResultEntry(jobId)); } @Test @@ -249,11 +257,28 @@ public void testCleanupAfterLeadershipChange() throws Exception { // Construct job graph store. final AtomicInteger actualGlobalCleanupCallCount = new AtomicInteger(); - final OneShotLatch successfulCleanupLatch = new OneShotLatch(); - final RuntimeException temporaryError = new RuntimeException("Unable to remove job graph."); + final OneShotLatch firstCleanupTriggered = new OneShotLatch(); + final CompletableFuture successfulJobGraphCleanup = new CompletableFuture<>(); final JobGraphStore jobGraphStore = - createAndStartJobGraphStoreWithCleanupFailures( - 1, temporaryError, actualGlobalCleanupCallCount, successfulCleanupLatch); + TestingJobGraphStore.newBuilder() + .setGlobalCleanupFunction( + (actualJobId, ignoredExecutor) -> { + final int callCount = + actualGlobalCleanupCallCount.getAndIncrement(); + firstCleanupTriggered.trigger(); + + if (callCount < 1) { + return FutureUtils.completedExceptionally( + new RuntimeException( + "Expected RuntimeException: Unable to remove job graph.")); + } + + successfulJobGraphCleanup.complete(actualJobId); + return FutureUtils.completedVoidFuture(); + }) + .build(); + + jobGraphStore.start(NoOpJobGraphListener.INSTANCE); haServices.setJobGraphStore(jobGraphStore); // Construct leader election service. @@ -262,23 +287,10 @@ public void testCleanupAfterLeadershipChange() throws Exception { haServices.setJobMasterLeaderElectionService(jobId, leaderElectionService); // start the dispatcher with no retries on cleanup - final CountDownLatch jobGraphRemovalErrorReceived = new CountDownLatch(1); - final Dispatcher dispatcher = - createTestingDispatcherBuilder() - .setFatalErrorHandler( - throwable -> { - final Optional maybeError = - ExceptionUtils.findThrowable( - throwable, temporaryError::equals); - if (maybeError.isPresent()) { - jobGraphRemovalErrorReceived.countDown(); - } else { - testingFatalErrorHandlerResource - .getFatalErrorHandler() - .onFatalError(throwable); - } - }) - .build(); + configuration.set( + CleanupOptions.CLEANUP_STRATEGY, + CleanupOptions.NONE_PARAM_VALUES.iterator().next()); + final Dispatcher dispatcher = createTestingDispatcherBuilder().build(rpcService); dispatcher.start(); toTerminate.add(dispatcher); @@ -288,7 +300,7 @@ public void testCleanupAfterLeadershipChange() throws Exception { dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); waitForJobToFinish(leaderElectionService, dispatcherGateway, jobId); - jobGraphRemovalErrorReceived.await(); + firstCleanupTriggered.await(); // Remove job master leadership. leaderElectionService.notLeader(); @@ -296,32 +308,38 @@ public void testCleanupAfterLeadershipChange() throws Exception { // This will clear internal state of election service, so a new contender can register. leaderElectionService.stop(); - assertThat(successfulCleanupLatch.isTriggered(), CoreMatchers.is(false)); + assertThat( + "The cleanup should have been triggered only once.", + actualGlobalCleanupCallCount.get(), + equalTo(1)); + assertThat( + "The cleanup should not have reached the successful cleanup code path.", + successfulJobGraphCleanup.isDone(), + equalTo(false)); assertThat( "The JobGraph is still stored in the JobGraphStore.", haServices.getJobGraphStore().getJobIds(), - CoreMatchers.is(Collections.singleton(jobId))); + equalTo(Collections.singleton(jobId))); assertThat( - "The JobResultStore has this job marked as dirty.", + "The JobResultStore should have this job marked as dirty.", haServices.getJobResultStore().getDirtyResults().stream() .map(JobResult::getJobId) .collect(Collectors.toSet()), - CoreMatchers.is(Collections.singleton(jobId))); + equalTo(Collections.singleton(jobId))); // Run a second dispatcher, that restores our finished job. final Dispatcher secondDispatcher = createTestingDispatcherBuilder() .setRecoveredDirtyJobs(haServices.getJobResultStore().getDirtyResults()) - .build(); + .build(rpcService); secondDispatcher.start(); toTerminate.add(secondDispatcher); leaderElectionService.isLeader(UUID.randomUUID()); CommonTestUtils.waitUntilCondition( - () -> haServices.getJobResultStore().getDirtyResults().isEmpty(), - Deadline.fromNow(TimeUtils.toDuration(TIMEOUT))); + () -> haServices.getJobResultStore().getDirtyResults().isEmpty()); assertThat( "The JobGraph is not stored in the JobGraphStore.", @@ -331,38 +349,11 @@ public void testCleanupAfterLeadershipChange() throws Exception { "The JobResultStore has the job listed as clean.", haServices.getJobResultStore().hasJobResultEntry(jobId)); - // wait for the successful cleanup to be triggered - successfulCleanupLatch.await(); + assertThat(successfulJobGraphCleanup.get(), equalTo(jobId)); assertThat(actualGlobalCleanupCallCount.get(), equalTo(2)); } - private JobGraphStore createAndStartJobGraphStoreWithCleanupFailures( - int numberOfCleanupFailures, - Throwable throwable, - AtomicInteger actualCleanupCallCount, - OneShotLatch successfulCleanupLatch) - throws Exception { - final AtomicInteger failureCount = new AtomicInteger(numberOfCleanupFailures); - final JobGraphStore jobGraphStore = - TestingJobGraphStore.newBuilder() - .setGlobalCleanupFunction( - (ignoredJobId, ignoredExecutor) -> { - actualCleanupCallCount.incrementAndGet(); - - if (failureCount.getAndDecrement() > 0) { - return FutureUtils.completedExceptionally(throwable); - } - - successfulCleanupLatch.trigger(); - return FutureUtils.completedVoidFuture(); - }) - .build(); - - jobGraphStore.start(null); - return jobGraphStore; - } - private void waitForJobToFinish( TestingLeaderElectionService leaderElectionService, DispatcherGateway dispatcherGateway, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java index 73cf51a7bf6b8..c017e72f555a5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java @@ -165,7 +165,10 @@ private void startDispatcher( TestingDispatcher.Builder dispatcherBuilder, JobManagerRunnerFactory jobManagerRunnerFactory) throws Exception { - dispatcher = dispatcherBuilder.setJobManagerRunnerFactory(jobManagerRunnerFactory).build(); + dispatcher = + dispatcherBuilder + .setJobManagerRunnerFactory(jobManagerRunnerFactory) + .build(rpcService); dispatcher.start(); @@ -176,7 +179,6 @@ private TestingDispatcher.Builder createTestingDispatcherBuilder() { final JobManagerRunnerRegistry jobManagerRunnerRegistry = new DefaultJobManagerRunnerRegistry(2); return TestingDispatcher.builder() - .setRpcService(rpcService) .setBlobServer(blobServer) .setJobManagerRunnerRegistry(jobManagerRunnerRegistry) .setFatalErrorHandler(testingFatalErrorHandlerResource.getFatalErrorHandler()) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index a3d1666953847..b2476562b104f 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.operators.ResourceSpec; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.SavepointFormatType; @@ -87,7 +86,6 @@ import org.apache.flink.util.FlinkException; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.Preconditions; -import org.apache.flink.util.TimeUtils; import org.apache.flink.util.concurrent.FutureUtils; import org.assertj.core.api.Assertions; @@ -171,7 +169,7 @@ private TestingDispatcher createAndStartDispatcher( .setJobManagerRunnerFactory(jobManagerRunnerFactory) .setJobGraphWriter(haServices.getJobGraphStore()) .setJobResultStore(haServices.getJobResultStore()) - .build(); + .build(rpcService); dispatcher.start(); return dispatcher; } @@ -253,7 +251,7 @@ public void testDuplicateJobSubmissionWithRunningJobId() throws Exception { new ExpectedJobIdJobManagerRunnerFactory( jobId, createdJobManagerRunnerLatch)) .setRecoveredJobs(Collections.singleton(jobGraph)) - .build(); + .build(rpcService); dispatcher.start(); final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); @@ -422,7 +420,7 @@ public void testCancellationOfCanceledTerminalDoesNotThrowException() throws Exc .build()))); // wait for job to finish - dispatcherGateway.requestJobResult(jobId, TIMEOUT).get(); + dispatcher.getJobTerminationFuture(jobId, TIMEOUT).get(); // sanity check assertThat( dispatcherGateway.requestJobStatus(jobId, TIMEOUT).get(), is(JobStatus.CANCELED)); @@ -456,7 +454,7 @@ public void testCancellationOfNonCanceledTerminalJobFailsWithAppropriateExceptio .build()))); // wait for job to finish - dispatcherGateway.requestJobResult(jobId, TIMEOUT).get(); + dispatcher.getJobTerminationFuture(jobId, TIMEOUT).get(); // sanity check assertThat( dispatcherGateway.requestJobStatus(jobId, TIMEOUT).get(), is(JobStatus.FINISHED)); @@ -485,7 +483,7 @@ public void testNoHistoryServerArchiveCreatedForSuspendedJob() throws Exception archiveAttemptFuture.complete(null); return CompletableFuture.completedFuture(null); }) - .build(); + .build(rpcService); dispatcher.start(); jobMasterLeaderElectionService.isLeader(UUID.randomUUID()); DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); @@ -543,7 +541,7 @@ public void testJobManagerRunnerInitializationFailureFailsJob() throws Exception testFailure)); // wait till job has failed - dispatcherGateway.requestJobResult(jobId, TIMEOUT).get(); + dispatcher.getJobTerminationFuture(jobId, TIMEOUT).get(); // get failure cause ArchivedExecutionGraph execGraph = @@ -680,7 +678,7 @@ public void testFatalErrorIfRecoveredJobsCannotBeStarted() throws Exception { createTestingDispatcherBuilder() .setJobManagerRunnerFactory(jobManagerRunnerFactory) .setRecoveredJobs(Collections.singleton(JobGraphTestUtils.emptyJobGraph())) - .build(); + .build(rpcService); dispatcher.start(); @@ -725,7 +723,7 @@ public void testThatDirtilyFinishedJobsNotBeingRetriggered() throws Exception { createTestingDispatcherBuilder() .setRecoveredJobs(Collections.singleton(jobGraph)) .setRecoveredDirtyJobs(Collections.singleton(jobResult)) - .build(); + .build(rpcService); } @Test @@ -754,7 +752,7 @@ public void testJobCleanupWithoutRecoveredJobGraph() throws Exception { dispatcherBootstrapLatch.trigger(); return new NoOpDispatcherBootstrap(); }) - .build(); + .build(rpcService); dispatcher.start(); @@ -781,7 +779,9 @@ public void testPersistedJobGraphWhenDispatcherIsShutDown() throws Exception { haServices.setJobGraphStore(submittedJobGraphStore); dispatcher = - createTestingDispatcherBuilder().setJobGraphWriter(submittedJobGraphStore).build(); + createTestingDispatcherBuilder() + .setJobGraphWriter(submittedJobGraphStore) + .build(rpcService); dispatcher.start(); @@ -854,7 +854,6 @@ public void testJobStatusIsShownDuringTermination() throws Exception { .getState(); return status == JobStatus.SUSPENDED; }, - Deadline.fromNow(TimeUtils.toDuration(TIMEOUT)), 5L); } finally { // Unblock the termination of the second job @@ -902,7 +901,7 @@ public void testOnRemovedJobGraphDoesNotCleanUpHAFiles() throws Exception { createTestingDispatcherBuilder() .setRecoveredJobs(Collections.singleton(jobGraph)) .setJobGraphWriter(testingJobGraphStore) - .build(); + .build(rpcService); dispatcher.start(); final CompletableFuture processFuture = @@ -950,7 +949,7 @@ public void testRequestMultipleJobDetails_returnsSuspendedJobs() throws Exceptio jobMasterLeaderElectionService.isLeader(UUID.randomUUID()); dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); - dispatcherGateway.requestJobResult(jobId, TIMEOUT).get(); + dispatcher.getJobTerminationFuture(jobId, TIMEOUT).get(); assertOnlyContainsSingleJobWithState( JobStatus.SUSPENDED, dispatcherGateway.requestMultipleJobDetails(TIMEOUT).get()); @@ -996,6 +995,8 @@ public void testRequestMultipleJobDetails_returnsFinishedOverSuspendedJob() thro // run second job, which completes with FINISHED dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); + dispatcher.getJobTerminationFuture(jobId, TIMEOUT).get(); + assertOnlyContainsSingleJobWithState( JobStatus.FINISHED, dispatcherGateway.requestMultipleJobDetails(TIMEOUT).get()); } @@ -1011,7 +1012,7 @@ public void testRequestMultipleJobDetails_isSerializable() throws Exception { jobMasterLeaderElectionService.isLeader(UUID.randomUUID()); dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); - dispatcherGateway.requestJobResult(jobId, TIMEOUT).get(); + dispatcher.getJobTerminationFuture(jobId, TIMEOUT).get(); final MultipleJobsDetails multipleJobsDetails = dispatcherGateway.requestMultipleJobDetails(TIMEOUT).get(); @@ -1071,7 +1072,7 @@ public void testOnlyRecoveredJobsAreRetainedInTheBlobServer() throws Exception { dispatcher = createTestingDispatcherBuilder() .setRecoveredJobs(Collections.singleton(new JobGraph(jobId1, "foobar"))) - .build(); + .build(rpcService); Assertions.assertThat(blobServer.getFile(jobId1, blobKey1)).hasBinaryContent(fileContent); Assertions.assertThatThrownBy(() -> blobServer.getFile(jobId2, blobKey2)) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobDispatcherITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobDispatcherITCase.java index 9be03d377b07e..f6fbcdde6bc1a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobDispatcherITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobDispatcherITCase.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.runtime.clusterframework.ApplicationStatus; @@ -59,7 +58,6 @@ import java.io.ObjectOutputStream; import java.nio.file.Files; import java.nio.file.Path; -import java.time.Duration; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; @@ -73,8 +71,6 @@ @ExtendWith(TestLoggerExtension.class) public class JobDispatcherITCase { - private static final Duration TIMEOUT = Duration.ofMinutes(10); - private Supplier createJobModeDispatcherResourceManagerComponentFactorySupplier( Configuration configuration) { @@ -95,7 +91,6 @@ public class JobDispatcherITCase { @Test public void testRecoverFromCheckpointAfterLosingAndRegainingLeadership(@TempDir Path tmpPath) throws Exception { - final Deadline deadline = Deadline.fromNow(TIMEOUT); final Configuration configuration = new Configuration(); configuration.set(HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name()); final TestingMiniClusterConfiguration clusterConfiguration = @@ -134,7 +129,7 @@ public void testRecoverFromCheckpointAfterLosingAndRegainingLeadership(@TempDir haServices.grantDispatcherLeadership(); // job is suspended, wait until it's running - awaitJobStatus(cluster, jobID, JobStatus.RUNNING, deadline); + awaitJobStatus(cluster, jobID, JobStatus.RUNNING); CommonTestUtils.waitUntilCondition( () -> @@ -142,8 +137,7 @@ public void testRecoverFromCheckpointAfterLosingAndRegainingLeadership(@TempDir .get() .getCheckpointStatsSnapshot() .getLatestRestoredCheckpoint() - != null, - deadline); + != null); } } @@ -174,8 +168,7 @@ private JobID generateAndPersistJobGraph( return jobGraph.getJobID(); } - private static void awaitJobStatus( - MiniCluster cluster, JobID jobId, JobStatus status, Deadline deadline) + private static void awaitJobStatus(MiniCluster cluster, JobID jobId, JobStatus status) throws Exception { CommonTestUtils.waitUntilCondition( () -> { @@ -189,8 +182,7 @@ private static void awaitJobStatus( } throw e; } - }, - deadline); + }); } /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java index 444c77e8a553f..9c18622d136cc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java @@ -41,7 +41,6 @@ import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; @@ -177,7 +176,6 @@ public static TestingDispatcher.Builder builder() { } public static class Builder { - private RpcService rpcService = new TestingRpcService(); private DispatcherId fencingToken = DispatcherId.generate(); private Collection recoveredJobs = Collections.emptyList(); @Nullable private Collection recoveredDirtyJobs = null; @@ -217,11 +215,6 @@ public static class Builder { new DefaultJobManagerRunnerRegistry(1); @Nullable private ResourceCleanerFactory resourceCleanerFactory; - public Builder setRpcService(RpcService rpcService) { - this.rpcService = rpcService; - return this; - } - public Builder setFencingToken(DispatcherId fencingToken) { this.fencingToken = fencingToken; return this; @@ -354,7 +347,7 @@ private ResourceCleanerFactory createDefaultResourceCleanerFactory() { jobManagerMetricGroup); } - public TestingDispatcher build() throws Exception { + public TestingDispatcher build(RpcService rpcService) throws Exception { return new TestingDispatcher( rpcService, fencingToken, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunnerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunnerTest.java index 8a226fed8aa7e..cedcfc3d43510 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunnerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunnerTest.java @@ -32,6 +32,7 @@ import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.dispatcher.UnavailableDispatcherOperationException; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.jobmaster.JobManagerRunnerResult; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; @@ -622,7 +623,8 @@ public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, SharedStateRegistryFactory sharedStateRegistryFactory, - Executor ioExecutor) + Executor ioExecutor, + RestoreMode restoreMode) throws Exception { creationLatch.await(); return completedCheckpointStore; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactoryTest.java index ed6ef75a5e6bf..cd3d0a573b746 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactoryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactoryTest.java @@ -20,6 +20,7 @@ import org.apache.flink.configuration.CleanupOptions; import org.apache.flink.configuration.Configuration; +import org.apache.flink.util.Preconditions; import org.apache.flink.util.concurrent.ExponentialBackoffRetryStrategy; import org.apache.flink.util.concurrent.FixedRetryStrategy; import org.apache.flink.util.concurrent.RetryStrategy; @@ -64,7 +65,7 @@ public void testDefaultStrategyCreation() { new Configuration(), CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_INITIAL_BACKOFF.defaultValue(), CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_BACKOFF.defaultValue(), - Integer.MAX_VALUE); + CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_ATTEMPTS.defaultValue()); } private static Configuration createConfigurationWithRetryStrategy(String configValue) { @@ -115,8 +116,11 @@ public void testFixedDelayStrategyWithCustomDelay() { public void testFixedDelayStrategyWithCustomMaxAttempts() { final Configuration config = createConfigurationWithRetryStrategy(CleanupOptions.FIXED_DELAY_LABEL); - final int customMaxAttempts = - CleanupOptions.CLEANUP_STRATEGY_FIXED_DELAY_ATTEMPTS.defaultValue() + 2; + final int customMaxAttempts = 1; + Preconditions.checkArgument( + customMaxAttempts + != CleanupOptions.CLEANUP_STRATEGY_FIXED_DELAY_ATTEMPTS.defaultValue(), + "The custom value should be different from the default value to make it possible that the overwritten value is selected."); config.set(CleanupOptions.CLEANUP_STRATEGY_FIXED_DELAY_ATTEMPTS, customMaxAttempts); testFixedDelayStrategyCreation( @@ -158,7 +162,7 @@ private static void testExponentialBackoffDelayRetryStrategyWithDefaultValues(St config, CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_INITIAL_BACKOFF.defaultValue(), CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_BACKOFF.defaultValue(), - Integer.MAX_VALUE); + CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_ATTEMPTS.defaultValue()); } @Test @@ -176,7 +180,7 @@ public void testExponentialBackoffDelayRetryStrategyWithCustomMinimumDelay() { config, customMinDelay, CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_BACKOFF.defaultValue(), - Integer.MAX_VALUE); + CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_ATTEMPTS.defaultValue()); } @Test @@ -193,7 +197,7 @@ public void testExponentialBackoffDelayRetryStrategyWithCustomMaximumDelay() { config, CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_INITIAL_BACKOFF.defaultValue(), customMaxDelay, - Integer.MAX_VALUE); + CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_ATTEMPTS.defaultValue()); } @Test @@ -203,12 +207,17 @@ public void testExponentialBackoffDelayRetryStrategyWithCustomMaxAttempts() { // 13 is the minimum we can use for this test; otherwise, assertMaxDelay would fail due to a // Precondition in ExponentialBackoffRetryStrategy final int customMaxAttempts = 13; + Preconditions.checkArgument( + customMaxAttempts + != CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_ATTEMPTS + .defaultValue(), + "The custom value should be different from the default value to make it possible that the overwritten value is selected."); config.set( CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_ATTEMPTS, customMaxAttempts); testExponentialBackoffDelayRetryStrategyCreation( config, - CleanupOptions.CLEANUP_STRATEGY_FIXED_DELAY_DELAY.defaultValue(), + CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_INITIAL_BACKOFF.defaultValue(), CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_BACKOFF.defaultValue(), customMaxAttempts); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java index f48ca0697be9d..a90a245a7c2a6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java @@ -46,6 +46,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; +import java.time.Duration; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -237,6 +238,71 @@ private void testJobRecovery( } } + @Test + public void testRecoveryWhileJobGraphRecoveryIsScheduledConcurrently() throws Exception { + final JobResult dirtyJobResult = + TestingJobResultStore.createSuccessfulJobResult(new JobID()); + + OneShotLatch recoveryInitiatedLatch = new OneShotLatch(); + OneShotLatch jobGraphAddedLatch = new OneShotLatch(); + + jobGraphStore = + TestingJobGraphStore.newBuilder() + // mimic behavior when recovering a JobGraph that is marked for deletion + .setRecoverJobGraphFunction((jobId, jobs) -> null) + .build(); + + jobResultStore = + TestingJobResultStore.builder() + .withGetDirtyResultsSupplier( + () -> { + recoveryInitiatedLatch.trigger(); + try { + jobGraphAddedLatch.await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + return Collections.singleton(dirtyJobResult); + }) + .build(); + + final CompletableFuture> recoveredJobGraphsFuture = + new CompletableFuture<>(); + final CompletableFuture> recoveredDirtyJobResultsFuture = + new CompletableFuture<>(); + dispatcherServiceFactory = + (ignoredDispatcherId, + recoveredJobs, + recoveredDirtyJobResults, + ignoredJobGraphWriter, + ignoredJobResultStore) -> { + recoveredJobGraphsFuture.complete(recoveredJobs); + recoveredDirtyJobResultsFuture.complete(recoveredDirtyJobResults); + return TestingDispatcherGatewayService.newBuilder().build(); + }; + + try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = + createDispatcherLeaderProcess()) { + dispatcherLeaderProcess.start(); + + // start returns without the initial recovery being completed + // mimic ZK message about an added jobgraph while the recovery is ongoing + recoveryInitiatedLatch.await(); + dispatcherLeaderProcess.onAddedJobGraph(dirtyJobResult.getJobId()); + jobGraphAddedLatch.trigger(); + + assertThat(recoveredJobGraphsFuture) + .succeedsWithin(Duration.ofHours(1)) + .satisfies(recovedJobGraphs -> assertThat(recovedJobGraphs).isEmpty()); + assertThat(recoveredDirtyJobResultsFuture) + .succeedsWithin(Duration.ofHours(1)) + .satisfies( + recoveredDirtyJobResults -> + assertThat(recoveredDirtyJobResults) + .containsExactly(dirtyJobResult)); + } + } + @Test public void closeAsync_stopsJobGraphStoreAndDispatcher() throws Exception { final CompletableFuture jobGraphStopFuture = new CompletableFuture<>(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java index 1617a3cc629bb..dde393d4fa782 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.dispatcher.runner; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; @@ -71,7 +70,6 @@ import java.io.File; import java.io.IOException; -import java.time.Duration; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ForkJoinPool; @@ -88,8 +86,6 @@ public class ZooKeeperDefaultDispatcherRunnerTest extends TestLogger { private static final Time TESTING_TIMEOUT = Time.seconds(10L); - private static final Duration VERIFICATION_TIMEOUT = Duration.ofSeconds(10L); - @ClassRule public static ZooKeeperResource zooKeeperResource = new ZooKeeperResource(); @ClassRule public static TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -225,9 +221,7 @@ public JobResultStore createJobResultStore() { final JobGraphStore submittedJobGraphStore = createZooKeeperJobGraphStore(client); CommonTestUtils.waitUntilCondition( - () -> submittedJobGraphStore.getJobIds().isEmpty(), - Deadline.fromNow(VERIFICATION_TIMEOUT), - 20L); + () -> submittedJobGraphStore.getJobIds().isEmpty(), 20L); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexTest.java index f39460d318fb8..26dc6bab119e6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexTest.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.executiongraph; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; @@ -106,9 +108,13 @@ public void testFindLatestAllocationIgnoresFailedAttempts() throws Exception { final JobGraph jobGraph = JobGraphTestUtils.streamingJobGraph(source); final TestingPhysicalSlotProvider withLimitedAmountOfPhysicalSlots = TestingPhysicalSlotProvider.createWithLimitedAmountOfPhysicalSlots(1); + final Configuration configuration = new Configuration(); + // make sure that retrieving the last (al)location is independent from the history size + configuration.set(JobManagerOptions.MAX_ATTEMPTS_HISTORY_SIZE, 1); final SchedulerBase scheduler = SchedulerTestingUtils.newSchedulerBuilder( jobGraph, ComponentMainThreadExecutorServiceAdapter.forMainThread()) + .setJobMasterConfiguration(configuration) .setExecutionSlotAllocatorFactory( SchedulerTestingUtils.newSlotSharingExecutionSlotAllocatorFactory( withLimitedAmountOfPhysicalSlots)) @@ -130,15 +136,15 @@ public void testFindLatestAllocationIgnoresFailedAttempts() throws Exception { cancelExecution(firstExecution); sourceExecutionVertex.resetForNewExecution(); - assertThat(sourceExecutionVertex.findLatestPriorAllocation()).hasValue(allocationId); - assertThat(sourceExecutionVertex.findLatestPriorLocation()).hasValue(taskManagerLocation); + assertThat(sourceExecutionVertex.findLastAllocation()).hasValue(allocationId); + assertThat(sourceExecutionVertex.findLastLocation()).hasValue(taskManagerLocation); final Execution secondExecution = sourceExecutionVertex.getCurrentExecutionAttempt(); cancelExecution(secondExecution); sourceExecutionVertex.resetForNewExecution(); - assertThat(sourceExecutionVertex.findLatestPriorAllocation()).hasValue(allocationId); - assertThat(sourceExecutionVertex.findLatestPriorLocation()).hasValue(taskManagerLocation); + assertThat(sourceExecutionVertex.findLastAllocation()).hasValue(allocationId); + assertThat(sourceExecutionVertex.findLastLocation()).hasValue(taskManagerLocation); } private void cancelExecution(Execution execution) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStoreTestInternal.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStoreTestInternal.java index 3a2de32559d7c..20d6f8a4c2994 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStoreTestInternal.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStoreTestInternal.java @@ -51,10 +51,31 @@ public class FileSystemJobResultStoreTestInternal { @TempDir File temporaryFolder; + private Path basePath; + @BeforeEach public void setupTest() throws IOException { - Path path = new Path(temporaryFolder.toURI()); - fileSystemJobResultStore = new FileSystemJobResultStore(path.getFileSystem(), path, false); + basePath = new Path(temporaryFolder.toURI()); + fileSystemJobResultStore = + new FileSystemJobResultStore(basePath.getFileSystem(), basePath, false); + } + + @Test + public void testValidEntryPathCreation() { + final Path entryParent = + fileSystemJobResultStore.constructEntryPath("random-name").getParent(); + assertThat(entryParent) + .extracting(FileSystemJobResultStoreTestInternal::stripSucceedingSlash) + .isEqualTo(stripSucceedingSlash(basePath)); + } + + private static String stripSucceedingSlash(Path path) { + final String uriStr = path.toUri().toString(); + if (uriStr.charAt(uriStr.length() - 1) == '/') { + return uriStr.substring(0, uriStr.length() - 1); + } + + return uriStr; } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStoreTest.java index d45a39aeb25a4..1343e7f648375 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStoreTest.java @@ -41,8 +41,8 @@ import java.util.Collection; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; @@ -51,6 +51,7 @@ import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.fail; /** @@ -220,14 +221,22 @@ public void testGlobalCleanupWithNonExistName() throws Exception { .globalCleanupAsync(testingJobGraph.getJobID(), Executors.directExecutor()) .join(); - try { - removeFuture.get(timeout, TimeUnit.MILLISECONDS); - fail( - "We should get an expected timeout because we are removing a non-existed job graph."); - } catch (TimeoutException ex) { - // expected - } - assertThat(removeFuture.isDone(), is(false)); + assertThat(removeFuture.isDone(), is(true)); + } + + @Test + public void testGlobalCleanupFailsIfRemovalReturnsFalse() throws Exception { + final TestingStateHandleStore stateHandleStore = + builder.setRemoveFunction(name -> false).build(); + + final JobGraphStore jobGraphStore = createAndStartJobGraphStore(stateHandleStore); + assertThrows( + ExecutionException.class, + () -> + jobGraphStore + .globalCleanupAsync( + testingJobGraph.getJobID(), Executors.directExecutor()) + .get()); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperJobGraphStoreWatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperJobGraphStoreWatcherTest.java index fd19143e52369..7a8210c416345 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperJobGraphStoreWatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperJobGraphStoreWatcherTest.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.jobmanager; import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener; @@ -41,8 +40,6 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.time.Duration; - import static org.hamcrest.Matchers.contains; import static org.junit.Assert.assertThat; @@ -53,8 +50,6 @@ public class ZooKeeperJobGraphStoreWatcherTest extends TestLogger { @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - private static final Duration TIMEOUT = Duration.ofMillis(30 * 1000); - private Configuration configuration; private TestingJobGraphListener testingJobGraphListener; @@ -87,16 +82,14 @@ public void testJobGraphAddedAndRemovedShouldNotifyGraphStoreListener() throws E stateHandleStore.addAndLock("/" + jobID, jobGraph); CommonTestUtils.waitUntilCondition( - () -> testingJobGraphListener.getAddedJobGraphs().size() > 0, - Deadline.fromNow(TIMEOUT)); + () -> testingJobGraphListener.getAddedJobGraphs().size() > 0); assertThat(testingJobGraphListener.getAddedJobGraphs(), contains(jobID)); stateHandleStore.releaseAndTryRemove("/" + jobID); CommonTestUtils.waitUntilCondition( - () -> testingJobGraphListener.getRemovedJobGraphs().size() > 0, - Deadline.fromNow(TIMEOUT)); + () -> testingJobGraphListener.getRemovedJobGraphs().size() > 0); assertThat(testingJobGraphListener.getRemovedJobGraphs(), contains(jobID)); jobGraphStoreWatcher.stop(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java index 50a97001b19ab..f61e6690f480a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java @@ -24,7 +24,6 @@ import org.apache.flink.api.common.functions.AggregateFunction; import org.apache.flink.api.common.io.DefaultInputSplitAssigner; import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.ClosureCleaner; import org.apache.flink.api.java.tuple.Tuple3; @@ -98,7 +97,6 @@ import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; -import org.apache.flink.runtime.taskexecutor.TaskExecutorToJobManagerHeartbeatPayload; import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; @@ -116,7 +114,6 @@ import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.TestLogger; -import org.apache.flink.util.TimeUtils; import org.apache.flink.util.concurrent.FutureUtils; import org.hamcrest.Matchers; @@ -156,7 +153,6 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.BiConsumer; -import java.util.function.BiFunction; import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -780,7 +776,6 @@ public void testResourceManagerBecomesUnreachableTriggersDisconnect() throws Exc jobMasterGateway.heartbeatFromResourceManager(rmResourceId); return disconnectedJobManagerFuture.isDone(); }, - Deadline.fromNow(TimeUtils.toDuration(testingTimeout)), 50L); // heartbeat timeout should trigger disconnect JobManager from ResourceManager @@ -1179,8 +1174,6 @@ private Supplier getInputSplitSupplier( private void waitUntilAllExecutionsAreScheduledOrDeployed( final JobMasterGateway jobMasterGateway) throws Exception { - final Duration duration = Duration.ofMillis(testingTimeout.toMilliseconds()); - final Deadline deadline = Deadline.fromNow(duration); CommonTestUtils.waitUntilCondition( () -> { @@ -1192,8 +1185,7 @@ private void waitUntilAllExecutionsAreScheduledOrDeployed( execution.getState() == ExecutionState.SCHEDULED || execution.getState() == ExecutionState.DEPLOYING); - }, - deadline); + }); } private static AccessExecution getFirstExecution( @@ -1752,9 +1744,7 @@ public void testJobFailureWhenGracefulTaskExecutorTermination() throws Exception (localTaskManagerLocation, jobMasterGateway) -> jobMasterGateway.disconnectTaskManager( localTaskManagerLocation.getResourceID(), - new FlinkException("Test disconnectTaskManager exception.")), - (jobMasterGateway, resourceID) -> - (ignoredA, ignoredB) -> FutureUtils.completedVoidFuture()); + new FlinkException("Test disconnectTaskManager exception."))); } @Test @@ -1766,14 +1756,7 @@ public void testJobFailureWhenTaskExecutorHeartbeatTimeout() throws Exception { testingHeartbeatService, (localTaskManagerLocation, jobMasterGateway) -> testingHeartbeatService.triggerHeartbeatTimeout( - jmResourceId, localTaskManagerLocation.getResourceID()), - (jobMasterGateway, taskManagerResourceId) -> - (resourceId, ignored) -> { - jobMasterGateway.heartbeatFromTaskManager( - taskManagerResourceId, - TaskExecutorToJobManagerHeartbeatPayload.empty()); - return FutureUtils.completedVoidFuture(); - }); + jmResourceId, localTaskManagerLocation.getResourceID())); } /** @@ -1957,8 +1940,7 @@ public void testJobMasterAcceptsSlotsWhenJobIsRestarting() throws Exception { CommonTestUtils.waitUntilCondition( () -> jobMasterGateway.requestJobStatus(testingTimeout).get() - == JobStatus.RUNNING, - Deadline.fromNow(TimeUtils.toDuration(testingTimeout))); + == JobStatus.RUNNING); jobMasterGateway.disconnectTaskManager( unresolvedTaskManagerLocation.getResourceID(), @@ -1967,8 +1949,7 @@ public void testJobMasterAcceptsSlotsWhenJobIsRestarting() throws Exception { CommonTestUtils.waitUntilCondition( () -> jobMasterGateway.requestJobStatus(testingTimeout).get() - == JobStatus.RESTARTING, - Deadline.fromNow(TimeUtils.toDuration(testingTimeout))); + == JobStatus.RESTARTING); assertThat( registerSlotsAtJobMaster( @@ -1987,12 +1968,7 @@ public void testJobMasterAcceptsSlotsWhenJobIsRestarting() throws Exception { private void runJobFailureWhenTaskExecutorTerminatesTest( HeartbeatServices heartbeatServices, - BiConsumer jobReachedRunningState, - BiFunction< - JobMasterGateway, - ResourceID, - BiFunction>> - heartbeatConsumerFunction) + BiConsumer jobReachedRunningState) throws Exception { final JobGraph jobGraph = JobGraphTestUtils.singleNoOpJobGraph(); final JobMasterBuilder.TestingOnCompletionActions onCompletionActions = @@ -2024,10 +2000,6 @@ private void runJobFailureWhenTaskExecutorTerminatesTest( taskDeploymentDescriptor.getExecutionAttemptId()); return CompletableFuture.completedFuture(Acknowledge.get()); }) - .setHeartbeatJobManagerFunction( - heartbeatConsumerFunction.apply( - jobMasterGateway, - taskManagerUnresolvedLocation.getResourceID())) .createTestingTaskExecutorGateway(); final Collection slotOffers = diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolInteractionsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolInteractionsTest.java index b98029cdbc974..0406e9b34bd3a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolInteractionsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolInteractionsTest.java @@ -18,13 +18,11 @@ package org.apache.flink.runtime.jobmaster.slotpool; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.executiongraph.TestingComponentMainThreadExecutor; import org.apache.flink.runtime.jobmaster.SlotRequestId; import org.apache.flink.runtime.testutils.CommonTestUtils; -import org.apache.flink.testutils.TestingUtils; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; @@ -96,9 +94,7 @@ public void testCancelSlotAllocationWithoutResourceManager() throws Exception { assertTrue(ExceptionUtils.stripExecutionException(e) instanceof TimeoutException); } - CommonTestUtils.waitUntilCondition( - () -> pool.getNumPendingRequests() == 0, - Deadline.fromNow(TestingUtils.TESTING_DURATION)); + CommonTestUtils.waitUntilCondition(() -> pool.getNumPendingRequests() == 0); } } @@ -123,9 +119,7 @@ public void testSlotAllocationTimeout() throws Exception { assertTrue(ExceptionUtils.stripExecutionException(e) instanceof TimeoutException); } - CommonTestUtils.waitUntilCondition( - () -> pool.getNumPendingRequests() == 0, - Deadline.fromNow(TestingUtils.TESTING_DURATION)); + CommonTestUtils.waitUntilCondition(() -> pool.getNumPendingRequests() == 0); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionServiceTest.java index 23c689a33d401..17eb19f1b4311 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionServiceTest.java @@ -27,19 +27,16 @@ import org.junit.Test; import java.util.UUID; -import java.util.concurrent.TimeoutException; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; import static org.junit.Assert.assertThat; -import static org.junit.Assert.fail; /** Tests for {@link DefaultLeaderElectionService}. */ public class DefaultLeaderElectionServiceTest extends TestLogger { private static final String TEST_URL = "akka//user/jobmanager"; - private static final long timeout = 50L; @Test public void testOnGrantAndRevokeLeadership() throws Exception { @@ -50,7 +47,7 @@ public void testOnGrantAndRevokeLeadership() throws Exception { // grant leadership testingLeaderElectionDriver.isLeader(); - testingContender.waitForLeader(timeout); + testingContender.waitForLeader(); assertThat(testingContender.getDescription(), is(TEST_URL)); assertThat( testingContender.getLeaderSessionID(), @@ -65,7 +62,7 @@ public void testOnGrantAndRevokeLeadership() throws Exception { // revoke leadership testingLeaderElectionDriver.notLeader(); - testingContender.waitForRevokeLeader(timeout); + testingContender.waitForRevokeLeader(); assertThat(testingContender.getLeaderSessionID(), is(nullValue())); assertThat(leaderElectionService.getLeaderSessionID(), is(nullValue())); // External storage should be cleared @@ -238,7 +235,7 @@ public void testErrorForwarding() throws Exception { testingLeaderElectionDriver.onFatalError(testException); - testingContender.waitForError(timeout); + testingContender.waitForError(); assertThat(testingContender.getError(), is(notNullValue())); assertThat( testingContender.getError(), @@ -258,15 +255,6 @@ public void testErrorIsIgnoredAfterBeingStop() throws Exception { leaderElectionService.stop(); testingLeaderElectionDriver.onFatalError(testException); - - try { - testingContender.waitForError(timeout); - fail( - "We expect to have a timeout here because there's no error should be passed to contender."); - } catch (TimeoutException ex) { - // noop - } - assertThat(testingContender.getError(), is(nullValue())); }); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeClusterComponentsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeClusterComponentsTest.java index e0fcb72c67f84..f25b13069452f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeClusterComponentsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeClusterComponentsTest.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobSubmissionResult; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServicesWithLeadershipControl; @@ -155,8 +154,7 @@ public void testReelectionOfJobMaster() throws Exception { @Test public void testTaskExecutorsReconnectToClusterWithLeadershipChange() throws Exception { - final Deadline deadline = Deadline.fromNow(TESTING_TIMEOUT); - waitUntilTaskExecutorsHaveConnected(NUM_TMS, deadline); + waitUntilTaskExecutorsHaveConnected(NUM_TMS); highAvailabilityServices.revokeResourceManagerLeadership().get(); highAvailabilityServices.grantResourceManagerLeadership(); @@ -168,16 +166,14 @@ public void testTaskExecutorsReconnectToClusterWithLeadershipChange() throws Exc .getLeaderSessionId(), is(notNullValue())); - waitUntilTaskExecutorsHaveConnected(NUM_TMS, deadline); + waitUntilTaskExecutorsHaveConnected(NUM_TMS); } - private void waitUntilTaskExecutorsHaveConnected(int numTaskExecutors, Deadline deadline) - throws Exception { + private void waitUntilTaskExecutorsHaveConnected(int numTaskExecutors) throws Exception { CommonTestUtils.waitUntilCondition( () -> miniCluster.requestClusterOverview().get().getNumTaskManagersConnected() == numTaskExecutors, - deadline, 10L); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElectionTest.java index 82f3893494f4f..ae715a3ee8397 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElectionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElectionTest.java @@ -46,13 +46,13 @@ public void testStandaloneLeaderElectionRetrieval() throws Exception { leaderElectionService.start(contender); leaderRetrievalService.start(testingListener); - contender.waitForLeader(1000l); + contender.waitForLeader(); assertTrue(contender.isLeader()); assertEquals( HighAvailabilityServices.DEFAULT_LEADER_ID, contender.getLeaderSessionID()); - testingListener.waitForNewLeader(1000l); + testingListener.waitForNewLeader(); assertEquals(TEST_URL, testingListener.getAddress()); assertEquals( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderBase.java index a18019f52ea6c..f11cd58572202 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderBase.java @@ -18,16 +18,13 @@ package org.apache.flink.runtime.leaderelection; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.util.ExceptionUtils; import javax.annotation.Nullable; -import java.time.Duration; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.TimeUnit; /** * Base class which provides some convenience functions for testing purposes of {@link @@ -41,47 +38,32 @@ public class TestingLeaderBase { private boolean isLeader = false; private Throwable error; - public void waitForLeader(long timeout) throws Exception { + public void waitForLeader() throws Exception { throwExceptionIfNotNull(); - final String errorMsg = "Contender was not elected as the leader within " + timeout + "ms"; CommonTestUtils.waitUntilCondition( () -> { - final LeaderInformation leader = - leaderEventQueue.poll(timeout, TimeUnit.MILLISECONDS); - return leader != null && !leader.isEmpty(); - }, - Deadline.fromNow(Duration.ofMillis(timeout)), - errorMsg); + final LeaderInformation leader = leaderEventQueue.take(); + return !leader.isEmpty(); + }); isLeader = true; } - public void waitForRevokeLeader(long timeout) throws Exception { + public void waitForRevokeLeader() throws Exception { throwExceptionIfNotNull(); - final String errorMsg = "Contender was not revoked within " + timeout + "ms"; CommonTestUtils.waitUntilCondition( () -> { - final LeaderInformation leader = - leaderEventQueue.poll(timeout, TimeUnit.MILLISECONDS); - return leader != null && leader.isEmpty(); - }, - Deadline.fromNow(Duration.ofMillis(timeout)), - errorMsg); + final LeaderInformation leader = leaderEventQueue.take(); + return leader.isEmpty(); + }); isLeader = false; } - public void waitForError(long timeout) throws Exception { - final String errorMsg = "Contender did not see an exception with " + timeout + "ms"; - CommonTestUtils.waitUntilCondition( - () -> { - error = errorQueue.poll(timeout, TimeUnit.MILLISECONDS); - return error != null; - }, - Deadline.fromNow(Duration.ofMillis(timeout)), - errorMsg); + public void waitForError() throws Exception { + error = errorQueue.take(); } public void handleError(Throwable ex) { @@ -95,7 +77,7 @@ public void handleError(Throwable ex) { */ @Nullable public Throwable getError() { - return this.error; + return error == null ? errorQueue.poll() : error; } public boolean isLeader() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingRetrievalBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingRetrievalBase.java index 449eba9162939..958557ad2fb1e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingRetrievalBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingRetrievalBase.java @@ -18,18 +18,15 @@ package org.apache.flink.runtime.leaderelection; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.util.ExceptionUtils; import javax.annotation.Nullable; -import java.time.Duration; import java.util.UUID; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.TimeUnit; /** * Base class which provides some convenience functions for testing purposes of {@link @@ -45,51 +42,34 @@ public class TestingRetrievalBase { private String oldAddress; private Throwable error; - public String waitForNewLeader(long timeout) throws Exception { + public String waitForNewLeader() throws Exception { throwExceptionIfNotNull(); - final String errorMsg = - "Listener was not notified about a new leader within " + timeout + "ms"; CommonTestUtils.waitUntilCondition( () -> { - leader = leaderEventQueue.poll(timeout, TimeUnit.MILLISECONDS); - return leader != null - && !leader.isEmpty() - && !leader.getLeaderAddress().equals(oldAddress); - }, - Deadline.fromNow(Duration.ofMillis(timeout)), - errorMsg); + leader = leaderEventQueue.take(); + return !leader.isEmpty() && !leader.getLeaderAddress().equals(oldAddress); + }); oldAddress = leader.getLeaderAddress(); return leader.getLeaderAddress(); } - public void waitForEmptyLeaderInformation(long timeout) throws Exception { + public void waitForEmptyLeaderInformation() throws Exception { throwExceptionIfNotNull(); - final String errorMsg = - "Listener was not notified about an empty leader within " + timeout + "ms"; CommonTestUtils.waitUntilCondition( () -> { - leader = leaderEventQueue.poll(timeout, TimeUnit.MILLISECONDS); - return leader != null && leader.isEmpty(); - }, - Deadline.fromNow(Duration.ofMillis(timeout)), - errorMsg); + leader = leaderEventQueue.take(); + return leader.isEmpty(); + }); oldAddress = null; } - public void waitForError(long timeout) throws Exception { - final String errorMsg = "Listener did not see an exception with " + timeout + "ms"; - CommonTestUtils.waitUntilCondition( - () -> { - error = errorQueue.poll(timeout, TimeUnit.MILLISECONDS); - return error != null; - }, - Deadline.fromNow(Duration.ofMillis(timeout)), - errorMsg); + public void waitForError() throws Exception { + error = errorQueue.take(); } public void handleError(Throwable ex) { @@ -123,7 +103,7 @@ public int getLeaderEventQueueSize() { */ @Nullable public Throwable getError() { - return this.error; + return error == null ? errorQueue.poll() : error; } private void throwExceptionIfNotNull() throws Exception { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java index f6ae64ba89f4e..c101c3dc49212 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java @@ -69,7 +69,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -164,12 +163,12 @@ public void testZooKeeperLeaderElectionRetrieval() throws Exception { .createLeaderRetrievalDriver( retrievalEventHandler, retrievalEventHandler::handleError); - electionEventHandler.waitForLeader(timeout); + electionEventHandler.waitForLeader(); final LeaderInformation confirmedLeaderInformation = electionEventHandler.getConfirmedLeaderInformation(); assertThat(confirmedLeaderInformation.getLeaderAddress(), is(LEADER_ADDRESS)); - retrievalEventHandler.waitForNewLeader(timeout); + retrievalEventHandler.waitForNewLeader(); assertThat( retrievalEventHandler.getLeaderSessionID(), @@ -233,7 +232,7 @@ public void testZooKeeperReelection() throws Exception { while (deadline.hasTimeLeft() && numberSeenLeaders < num) { LOG.debug("Wait for new leader #{}.", numberSeenLeaders); - String address = listener.waitForNewLeader(deadline.timeLeft().toMillis()); + String address = listener.waitForNewLeader(); Matcher m = regex.matcher(address); @@ -320,7 +319,7 @@ public void testZooKeeperReelectionWithReplacement() throws Exception { Pattern regex = Pattern.compile(pattern); for (int i = 0; i < numTries; i++) { - listener.waitForNewLeader(timeout); + listener.waitForNewLeader(); String address = listener.getAddress(); @@ -388,7 +387,7 @@ public void testLeaderShouldBeCorrectedWhenOverwritten() throws Exception { createAndInitLeaderElectionDriver( curatorFrameworkWrapper.asCuratorFramework(), electionEventHandler); - electionEventHandler.waitForLeader(timeout); + electionEventHandler.waitForLeader(); final LeaderInformation confirmedLeaderInformation = electionEventHandler.getConfirmedLeaderInformation(); assertThat(confirmedLeaderInformation.getLeaderAddress(), is(LEADER_ADDRESS)); @@ -436,8 +435,8 @@ public void testLeaderShouldBeCorrectedWhenOverwritten() throws Exception { .createLeaderRetrievalDriver( retrievalEventHandler, retrievalEventHandler::handleError); - if (retrievalEventHandler.waitForNewLeader(timeout).equals(faultyContenderUrl)) { - retrievalEventHandler.waitForNewLeader(timeout); + if (retrievalEventHandler.waitForNewLeader().equals(faultyContenderUrl)) { + retrievalEventHandler.waitForNewLeader(); } assertThat( @@ -491,7 +490,7 @@ public void testExceptionForwarding() throws Exception { leaderElectionDriver = createAndInitLeaderElectionDriver(client, electionEventHandler); - electionEventHandler.waitForError(timeout); + electionEventHandler.waitForError(); assertNotNull(electionEventHandler.getError()); assertThat( @@ -556,9 +555,9 @@ public void testEphemeralZooKeeperNodes() throws Exception { cache.getListenable().addListener(existsListener); cache.start(); - electionEventHandler.waitForLeader(timeout); + electionEventHandler.waitForLeader(); - retrievalEventHandler.waitForNewLeader(timeout); + retrievalEventHandler.waitForNewLeader(); Future existsFuture = existsListener.nodeExists(); @@ -576,15 +575,7 @@ public void testEphemeralZooKeeperNodes() throws Exception { // make sure that the leader node has been deleted deletedFuture.get(timeout, TimeUnit.MILLISECONDS); - try { - retrievalEventHandler.waitForNewLeader(1000L); - - fail( - "TimeoutException was expected because there is no leader registered and " - + "thus there shouldn't be any leader information in ZooKeeper."); - } catch (TimeoutException e) { - // that was expected - } + retrievalEventHandler.waitForEmptyLeaderInformation(); } finally { electionEventHandler.close(); if (leaderRetrievalDriver != null) { @@ -616,14 +607,14 @@ public void testNotLeaderShouldNotCleanUpTheLeaderInformation() throws Exception createAndInitLeaderElectionDriver( curatorFrameworkWrapper.asCuratorFramework(), electionEventHandler); - electionEventHandler.waitForLeader(timeout); + electionEventHandler.waitForLeader(); final LeaderInformation confirmedLeaderInformation = electionEventHandler.getConfirmedLeaderInformation(); assertThat(confirmedLeaderInformation.getLeaderAddress(), is(LEADER_ADDRESS)); // Leader is revoked leaderElectionDriver.notLeader(); - electionEventHandler.waitForRevokeLeader(timeout); + electionEventHandler.waitForRevokeLeader(); assertThat( electionEventHandler.getConfirmedLeaderInformation(), is(LeaderInformation.empty())); @@ -634,7 +625,7 @@ public void testNotLeaderShouldNotCleanUpTheLeaderInformation() throws Exception .createLeaderRetrievalDriver( retrievalEventHandler, retrievalEventHandler::handleError); - retrievalEventHandler.waitForNewLeader(timeout); + retrievalEventHandler.waitForNewLeader(); assertThat( retrievalEventHandler.getLeaderSessionID(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java index cc0a62cd05046..80325506733de 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java @@ -112,7 +112,7 @@ public void testPublishLeaderInformation() throws Exception { leaderElectionDriver.publishLeaderInformation( componentId, leaderInformation); - leaderRetrievalListener.waitForNewLeader(10_000L); + leaderRetrievalListener.waitForNewLeader(); assertThat(leaderRetrievalListener.getLeader()) .isEqualTo(leaderInformation); @@ -146,12 +146,12 @@ public void testPublishEmptyLeaderInformation() throws Exception { componentId, LeaderInformation.known(UUID.randomUUID(), "foobar")); - leaderRetrievalListener.waitForNewLeader(10_000L); + leaderRetrievalListener.waitForNewLeader(); leaderElectionDriver.publishLeaderInformation( componentId, LeaderInformation.empty()); - leaderRetrievalListener.waitForEmptyLeaderInformation(10_000L); + leaderRetrievalListener.waitForEmptyLeaderInformation(); assertThat(leaderRetrievalListener.getLeader()) .isEqualTo(LeaderInformation.empty()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalServiceTest.java index e2e83aa985b13..0e751e4db29e6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalServiceTest.java @@ -28,19 +28,16 @@ import org.junit.Test; import java.util.UUID; -import java.util.concurrent.TimeoutException; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; import static org.junit.Assert.assertThat; -import static org.junit.Assert.fail; /** Tests for {@link DefaultLeaderElectionService}. */ public class DefaultLeaderRetrievalServiceTest extends TestLogger { private static final String TEST_URL = "akka//user/jobmanager"; - private static final long timeout = 50L; @Test public void testNotifyLeaderAddress() throws Exception { @@ -51,7 +48,7 @@ public void testNotifyLeaderAddress() throws Exception { final LeaderInformation newLeader = LeaderInformation.known(UUID.randomUUID(), TEST_URL); testingLeaderRetrievalDriver.onUpdate(newLeader); - testingListener.waitForNewLeader(timeout); + testingListener.waitForNewLeader(); assertThat( testingListener.getLeaderSessionID(), is(newLeader.getLeaderSessionID())); @@ -71,10 +68,10 @@ public void testNotifyLeaderAddressEmpty() throws Exception { final LeaderInformation newLeader = LeaderInformation.known(UUID.randomUUID(), TEST_URL); testingLeaderRetrievalDriver.onUpdate(newLeader); - testingListener.waitForNewLeader(timeout); + testingListener.waitForNewLeader(); testingLeaderRetrievalDriver.onUpdate(LeaderInformation.empty()); - testingListener.waitForEmptyLeaderInformation(timeout); + testingListener.waitForEmptyLeaderInformation(); assertThat(testingListener.getLeaderSessionID(), is(nullValue())); assertThat(testingListener.getAddress(), is(nullValue())); }); @@ -92,7 +89,7 @@ public void testErrorForwarding() throws Exception { testingLeaderRetrievalDriver.onFatalError(testException); - testingListener.waitForError(timeout); + testingListener.waitForError(); assertThat( testingListener.getError(), FlinkMatchers.containsCause(testException)); @@ -112,13 +109,6 @@ public void testErrorIsIgnoredAfterBeingStop() throws Exception { leaderRetrievalService.stop(); testingLeaderRetrievalDriver.onFatalError(testException); - try { - testingListener.waitForError(timeout); - fail( - "We expect to have a timeout here because there's no error should be passed to listener."); - } catch (TimeoutException ex) { - // noop - } assertThat(testingListener.getError(), is(nullValue())); }); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/SettableLeaderRetrievalServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/SettableLeaderRetrievalServiceTest.java index 9ca315324f8f1..c2e329ef12b85 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/SettableLeaderRetrievalServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/SettableLeaderRetrievalServiceTest.java @@ -25,8 +25,6 @@ import org.junit.Before; import org.junit.Test; -import java.time.Duration; - import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; @@ -34,7 +32,6 @@ public class SettableLeaderRetrievalServiceTest extends TestLogger { private SettableLeaderRetrievalService settableLeaderRetrievalService; - private static final Duration TIMEOUT = Duration.ofHours(1); @Before public void setUp() { @@ -50,7 +47,7 @@ public void testNotifyListenerLater() throws Exception { final TestingListener listener = new TestingListener(); settableLeaderRetrievalService.start(listener); - listener.waitForNewLeader(TIMEOUT.toMillis()); + listener.waitForNewLeader(); assertThat(listener.getAddress(), equalTo(localhost)); assertThat( listener.getLeaderSessionID(), equalTo(HighAvailabilityServices.DEFAULT_LEADER_ID)); @@ -65,7 +62,7 @@ public void testNotifyListenerImmediately() throws Exception { settableLeaderRetrievalService.notifyListener( localhost, HighAvailabilityServices.DEFAULT_LEADER_ID); - listener.waitForNewLeader(TIMEOUT.toMillis()); + listener.waitForNewLeader(); assertThat(listener.getAddress(), equalTo(localhost)); assertThat( listener.getLeaderSessionID(), equalTo(HighAvailabilityServices.DEFAULT_LEADER_ID)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalConnectionHandlingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalConnectionHandlingTest.java index f705103fb94ba..e8cda90a0635d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalConnectionHandlingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalConnectionHandlingTest.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.leaderretrieval; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.core.testutils.EachCallbackWrapper; import org.apache.flink.runtime.leaderelection.LeaderInformation; import org.apache.flink.runtime.testutils.CommonTestUtils; @@ -279,8 +278,7 @@ public void testNewLeaderAfterReconnectTriggersListenerNotification() throws Exc && afterConnectionReconnect .getLeaderAddress() .equals(newLeaderAddress); - }, - Deadline.fromNow(Duration.ofSeconds(30L))); + }); }); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/DescriptiveStatisticsHistogramTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/DescriptiveStatisticsHistogramTest.java index a919d70cb1294..f0a27b326b061 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/DescriptiveStatisticsHistogramTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/DescriptiveStatisticsHistogramTest.java @@ -19,10 +19,15 @@ package org.apache.flink.runtime.metrics; import org.apache.flink.metrics.AbstractHistogramTest; +import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.TestLoggerExtension; +import org.assertj.core.data.Offset; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; +import org.testcontainers.utility.ThrowingFunction; + +import static org.assertj.core.api.Assertions.assertThat; /** * Tests for {@link DescriptiveStatisticsHistogram} and {@link @@ -31,10 +36,58 @@ @ExtendWith(TestLoggerExtension.class) class DescriptiveStatisticsHistogramTest extends AbstractHistogramTest { + private static final double[] DATA = {1, 2, 3, 4, 5, 6, 7, 8, 9}; + /** Tests the histogram functionality of the DropwizardHistogramWrapper. */ @Test void testDescriptiveHistogram() { int size = 10; testHistogram(size, new DescriptiveStatisticsHistogram(size)); } + + /** Tests our workaround for https://issues.apache.org/jira/browse/MATH-1642. */ + @Test + void testSerialization() throws Exception { + testDuplication( + original -> { + final byte[] bytes = InstantiationUtil.serializeObject(original); + return (DescriptiveStatisticsHistogramStatistics.CommonMetricsSnapshot) + InstantiationUtil.deserializeObject(bytes, getClass().getClassLoader()); + }); + } + + @Test + void testCopy() throws Exception { + testDuplication(DescriptiveStatisticsHistogramStatistics.CommonMetricsSnapshot::copy); + } + + private static void testDuplication( + ThrowingFunction< + DescriptiveStatisticsHistogramStatistics.CommonMetricsSnapshot, + DescriptiveStatisticsHistogramStatistics.CommonMetricsSnapshot> + duplicator) + throws Exception { + + DescriptiveStatisticsHistogramStatistics.CommonMetricsSnapshot original = + new DescriptiveStatisticsHistogramStatistics.CommonMetricsSnapshot(); + original.evaluate(DATA); + + assertOperations(original); + + final DescriptiveStatisticsHistogramStatistics.CommonMetricsSnapshot copy = + duplicator.apply(original); + + assertOperations(copy); + } + + private static void assertOperations( + DescriptiveStatisticsHistogramStatistics.CommonMetricsSnapshot statistics) { + assertThat(statistics.getPercentile(0.5)).isEqualTo(1); + assertThat(statistics.getCount()).isEqualTo(9); + assertThat(statistics.getMin()).isEqualTo(1); + assertThat(statistics.getMax()).isEqualTo(9); + assertThat(statistics.getMean()).isEqualTo(5); + assertThat(statistics.getStandardDeviation()).isCloseTo(2.7, Offset.offset(0.5)); + assertThat(statistics.getValues()).containsExactly(DATA); + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java index a63a7527ae896..387d6679df685 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java @@ -107,6 +107,7 @@ import org.junit.ClassRule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; import java.time.Duration; import java.util.ArrayList; @@ -1608,7 +1609,8 @@ public void testCheckpointCleanerIsClosedAfterCheckpointServices() throws Except throw new RuntimeException(e); } }, - executorService); + executorService, + log); } finally { executorService.shutdownNow(); } @@ -1620,7 +1622,8 @@ public void testCheckpointCleanerIsClosedAfterCheckpointServices() throws Except */ public static void doTestCheckpointCleanerIsClosedAfterCheckpointServices( BiFunction schedulerFactory, - ScheduledExecutorService executorService) + ScheduledExecutorService executorService, + Logger logger) throws Exception { final CountDownLatch checkpointServicesShutdownBlocked = new CountDownLatch(1); final CountDownLatch cleanerClosed = new CountDownLatch(1); @@ -1638,9 +1641,17 @@ public void shutdown(JobStatus jobStatus, CheckpointsCleaner checkpointsCleaner) new StandaloneCheckpointIDCounter() { @Override - public void shutdown(JobStatus jobStatus) throws Exception { - checkpointServicesShutdownBlocked.await(); - super.shutdown(jobStatus); + public CompletableFuture shutdown(JobStatus jobStatus) { + try { + checkpointServicesShutdownBlocked.await(); + } catch (InterruptedException e) { + logger.error( + "An error occurred while executing waiting for the CheckpointServices shutdown.", + e); + Thread.currentThread().interrupt(); + } + + return super.shutdown(jobStatus); } }; final CheckpointsCleaner checkpointsCleaner = diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SchedulerUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SchedulerUtilsTest.java index dfc3b40da07f6..9e21dd42e0813 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SchedulerUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SchedulerUtilsTest.java @@ -32,6 +32,7 @@ import org.apache.flink.runtime.checkpoint.StandaloneCheckpointIDCounter; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.state.IncrementalRemoteKeyedStateHandle; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyedStateHandle; @@ -78,7 +79,8 @@ public void testSettingMaxNumberOfCheckpointsToRetain() throws Exception { new StandaloneCheckpointRecoveryFactory(), Executors.directExecutor(), log, - new JobID()); + new JobID(), + RestoreMode.CLAIM); assertEquals( maxNumberOfCheckpointsToRetain, @@ -104,7 +106,8 @@ public void testSharedStateRegistration() throws Exception { recoveryFactory, Executors.directExecutor(), log, - new JobID()); + new JobID(), + RestoreMode.CLAIM); SharedStateRegistry sharedStateRegistry = checkpointStore.getSharedStateRegistry(); @@ -122,12 +125,14 @@ public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, SharedStateRegistryFactory sharedStateRegistryFactory, - Executor ioExecutor) { + Executor ioExecutor, + RestoreMode restoreMode) { List checkpoints = singletonList(checkpoint); return new EmbeddedCompletedCheckpointStore( maxNumberOfCheckpointsToRetain, checkpoints, - sharedStateRegistryFactory.create(ioExecutor, checkpoints)); + sharedStateRegistryFactory.create( + ioExecutor, checkpoints, RestoreMode.DEFAULT)); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerClusterITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerClusterITCase.java index 272b022383f57..adc8a2ee6899e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerClusterITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerClusterITCase.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.WebOptions; @@ -188,8 +187,7 @@ public void testCheckpointStatsPersistedAcrossRescale() throws Exception { .getCounts() .getNumberOfCompletedCheckpoints() > 0) - .get(), - Deadline.fromNow(Duration.ofHours(1))); + .get()); miniCluster.terminateTaskManager(0); @@ -281,7 +279,6 @@ private void waitUntilParallelismForVertexReached( } return executionJobVertex.getParallelism() == targetParallelism; - }, - Deadline.fromNow(Duration.ofSeconds(10))); + }); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerSimpleITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerSimpleITCase.java index 992d2c47ab4a5..b4b66ed29e1e5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerSimpleITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerSimpleITCase.java @@ -22,7 +22,6 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.runtime.execution.Environment; @@ -46,7 +45,6 @@ import java.io.IOException; import java.time.Duration; -import java.time.temporal.ChronoUnit; import static org.junit.Assert.assertTrue; @@ -129,7 +127,6 @@ public void testJobCancellationWhileRestartingSucceeds() throws Exception { // wait until we are in RESTARTING state CommonTestUtils.waitUntilCondition( () -> miniCluster.getJobStatus(jobGraph.getJobID()).get() == JobStatus.RESTARTING, - Deadline.fromNow(Duration.of(timeInRestartingState, ChronoUnit.MILLIS)), 5); // now cancel while in RESTARTING state diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerTest.java index e673a288b2c48..6e6f6d3d60036 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerTest.java @@ -1462,7 +1462,8 @@ public void testCheckpointCleanerIsClosedAfterCheckpointServices() throws Except throw new RuntimeException(e); } }, - executorService); + executorService, + log); } finally { executorService.shutdownNow(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandlerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandlerImplTest.java index cab4abeb2a37a..f0c2bee92df90 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandlerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandlerImplTest.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.core.execution.SavepointFormatType; -import org.apache.flink.core.testutils.FlinkMatchers; import org.apache.flink.runtime.checkpoint.CheckpointProperties; import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; import org.apache.flink.runtime.checkpoint.TestingCheckpointScheduling; @@ -31,7 +30,6 @@ import org.apache.flink.runtime.state.testutils.EmptyStreamStateHandle; import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -152,10 +150,6 @@ public void testFailedTerminationHandling() throws ExecutionException, Interrupt createTestInstance(globalFailOverTriggered::complete); final ExecutionState expectedNonFinishedState = ExecutionState.FAILED; - final String expectedErrorMessage = - String.format( - "Inconsistent execution state after stopping with savepoint. At least one execution is still in one of the following states: %s. A global fail-over is triggered to recover the job %s.", - expectedNonFinishedState, JOB_ID); final EmptyStreamStateHandle streamStateHandle = new EmptyStreamStateHandle(); final CompletedCheckpoint completedSavepoint = createCompletedSavepoint(streamStateHandle); @@ -168,24 +162,14 @@ public void testFailedTerminationHandling() throws ExecutionException, Interrupt testInstance.getSavepointPath().get(); fail("An ExecutionException is expected."); } catch (Throwable e) { - final Optional actualFlinkException = - ExceptionUtils.findThrowable(e, FlinkException.class); + final Optional actualFlinkException = + ExceptionUtils.findThrowable(e, StopWithSavepointStoppingException.class); assertTrue( "A FlinkException should have been thrown.", actualFlinkException.isPresent()); - assertThat( - actualFlinkException.get(), - FlinkMatchers.containsMessage(expectedErrorMessage)); } assertTrue("Global fail-over was not triggered.", globalFailOverTriggered.isDone()); - assertThat( - globalFailOverTriggered.get(), FlinkMatchers.containsMessage(expectedErrorMessage)); - assertFalse("Savepoint should not be discarded.", streamStateHandle.isDisposed()); - - assertFalse( - "Checkpoint scheduling should not be enabled in case of failure.", - checkpointScheduling.isEnabled()); } @Test(expected = UnsupportedOperationException.class) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContextTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContextTest.java index d15f28e7705bd..380e65c1c7e32 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContextTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContextTest.java @@ -153,7 +153,7 @@ private void testAssignSplitToUnregisterdReader(boolean fromCoordinatorExecutor) } }, "assignSplits() should fail to assign the splits to a reader that is not registered.", - "Cannot assign splits"); + "Cannot assign splits " + splitsAssignment.assignment().get(0)); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskExecutorStateChangelogStoragesManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskExecutorStateChangelogStoragesManagerTest.java index d3f1767f5b32d..db3502aed4963 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskExecutorStateChangelogStoragesManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskExecutorStateChangelogStoragesManagerTest.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.state; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.operators.MailboxExecutor; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.StateChangelogOptions; import org.apache.flink.core.plugin.PluginManager; @@ -166,7 +167,7 @@ private static class TestStateChangelogStorage @Override public StateChangelogWriter createWriter( - String operatorID, KeyGroupRange keyGroupRange) { + String operatorID, KeyGroupRange keyGroupRange, MailboxExecutor mailboxExecutor) { return null; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/changelog/inmemory/StateChangelogStorageLoaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/changelog/inmemory/StateChangelogStorageLoaderTest.java index 6600ff4125d55..8fdfc26493b4f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/changelog/inmemory/StateChangelogStorageLoaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/changelog/inmemory/StateChangelogStorageLoaderTest.java @@ -17,6 +17,7 @@ package org.apache.flink.runtime.state.changelog.inmemory; +import org.apache.flink.api.common.operators.MailboxExecutor; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.StateChangelogOptions; import org.apache.flink.core.plugin.PluginManager; @@ -92,7 +93,7 @@ private static class TestStateChangelogStorage implements StateChangelogStorage { @Override public StateChangelogWriter createWriter( - String operatorID, KeyGroupRange keyGroupRange) { + String operatorID, KeyGroupRange keyGroupRange, MailboxExecutor mailboxExecutor) { return null; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/changelog/inmemory/StateChangelogStorageTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/changelog/inmemory/StateChangelogStorageTest.java index d5e33038070a9..4adde98878a9b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/changelog/inmemory/StateChangelogStorageTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/changelog/inmemory/StateChangelogStorageTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.state.changelog.inmemory; import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.mailbox.SyncMailboxExecutor; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.changelog.ChangelogStateHandle; import org.apache.flink.runtime.state.changelog.SequenceNumber; @@ -58,7 +59,11 @@ public class StateChangelogStorageTest { @Test(expected = IllegalStateException.class) public void testNoAppendAfterClose() throws IOException { StateChangelogWriter writer = - getFactory().createWriter(new OperatorID().toString(), KeyGroupRange.of(0, 0)); + getFactory() + .createWriter( + new OperatorID().toString(), + KeyGroupRange.of(0, 0), + new SyncMailboxExecutor()); writer.close(); writer.append(0, new byte[0]); } @@ -66,11 +71,12 @@ public void testNoAppendAfterClose() throws IOException { @Test public void testWriteAndRead() throws Exception { KeyGroupRange kgRange = KeyGroupRange.of(0, 5); - Map> appendsByKeyGroup = generateAppends(kgRange, 10, 20); + Map> appendsByKeyGroup = generateAppends(kgRange, 405, 20); try (StateChangelogStorage client = getFactory(); StateChangelogWriter writer = - client.createWriter(new OperatorID().toString(), kgRange)) { + client.createWriter( + new OperatorID().toString(), kgRange, new SyncMailboxExecutor())) { SequenceNumber prev = writer.initialSequenceNumber(); for (Map.Entry> entry : appendsByKeyGroup.entrySet()) { Integer group = entry.getKey(); @@ -78,6 +84,7 @@ public void testWriteAndRead() throws Exception { for (byte[] bytes : appends) { writer.append(group, bytes); } + writer.nextSequenceNumber(); } T handle = writer.persist(prev).get(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java index e980a51215b23..cbcc99c955650 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.AccessExecution; @@ -46,7 +45,6 @@ import org.junit.Test; import java.io.IOException; -import java.time.Duration; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.function.Predicate; @@ -58,7 +56,6 @@ /** Integration tests for the {@link TaskExecutor}. */ public class TaskExecutorITCase extends TestLogger { - private static final Duration TESTING_TIMEOUT = Duration.ofMinutes(2L); private static final int NUM_TMS = 2; private static final int SLOTS_PER_TM = 2; private static final int PARALLELISM = NUM_TMS * SLOTS_PER_TM; @@ -137,9 +134,7 @@ private CompletableFuture submitJobAndWaitUntilRunning(JobGraph jobGr assertThat(jobResultFuture.isDone(), is(false)); CommonTestUtils.waitUntilCondition( - jobIsRunning(() -> miniCluster.getExecutionGraph(jobGraph.getJobID())), - Deadline.fromNow(TESTING_TIMEOUT), - 50L); + jobIsRunning(() -> miniCluster.getExecutionGraph(jobGraph.getJobID())), 50L); return jobResultFuture; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index 2963e7f8e97a0..a2852c2a9738e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.resources.CPUResource; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.ConfigConstants; @@ -489,7 +488,6 @@ public void testResourceManagerBecomesUnreachableTriggersDisconnect() throws Exc taskExecutorGateway.heartbeatFromResourceManager(rmResourceId); return taskExecutorDisconnectFuture.isDone(); }, - Deadline.fromNow(TimeUtils.toDuration(timeout)), 50L)); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java index 7e01ee4a5d9b6..d36100be22eee 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java @@ -20,13 +20,14 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.checkpoint.CompletedCheckpointStats; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.executiongraph.AccessExecutionVertex; import org.apache.flink.runtime.executiongraph.ErrorInfo; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.messages.FlinkJobNotFoundException; import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.rest.messages.job.JobDetailsInfo; import org.apache.flink.util.FileUtils; @@ -41,18 +42,19 @@ import java.io.StringWriter; import java.lang.management.ManagementFactory; import java.lang.management.RuntimeMXBean; -import java.time.Duration; -import java.time.temporal.ChronoUnit; import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.function.Predicate; import java.util.stream.Stream; import static java.lang.String.format; +import static org.apache.flink.util.Preconditions.checkState; import static org.junit.jupiter.api.Assertions.fail; /** This class contains auxiliary methods for unit tests. */ @@ -131,40 +133,16 @@ public static void printLog4jDebugConfig(File file) throws IOException { } } - public static void waitUntilCondition( - SupplierWithException condition, Deadline timeout) - throws Exception { - waitUntilCondition(condition, timeout, RETRY_INTERVAL); - } - - public static void waitUntilCondition( - SupplierWithException condition, - Deadline timeout, - long retryIntervalMillis) - throws Exception { - waitUntilCondition( - condition, timeout, retryIntervalMillis, "Condition was not met in given timeout."); - } - - public static void waitUntilCondition( - SupplierWithException condition, Deadline timeout, String errorMsg) + public static void waitUntilCondition(SupplierWithException condition) throws Exception { - waitUntilCondition(condition, timeout, RETRY_INTERVAL, errorMsg); + waitUntilCondition(condition, RETRY_INTERVAL); } public static void waitUntilCondition( - SupplierWithException condition, - Deadline timeout, - long retryIntervalMillis, - String errorMsg) + SupplierWithException condition, long retryIntervalMillis) throws Exception { - while (timeout.hasTimeLeft() && !condition.get()) { - final long timeLeft = Math.max(0, timeout.timeLeft().toMillis()); - Thread.sleep(Math.min(retryIntervalMillis, timeLeft)); - } - - if (!timeout.hasTimeLeft()) { - throw new TimeoutException(errorMsg); + while (!condition.get()) { + Thread.sleep(retryIntervalMillis); } } @@ -182,17 +160,6 @@ public static void waitForAllTaskRunning( SupplierWithException executionGraphSupplier, boolean allowFinished) throws Exception { - waitForAllTaskRunning( - executionGraphSupplier, - Deadline.fromNow(Duration.of(1, ChronoUnit.MINUTES)), - allowFinished); - } - - public static void waitForAllTaskRunning( - SupplierWithException executionGraphSupplier, - Deadline timeout, - boolean allowFinished) - throws Exception { Predicate subtaskPredicate = task -> { switch (task.getExecutionState()) { @@ -225,13 +192,11 @@ public static void waitForAllTaskRunning( jobVertex -> Arrays.stream(jobVertex.getTaskVertices()) .allMatch(subtaskPredicate)); - }, - timeout); + }); } public static void waitForAllTaskRunning( - SupplierWithException jobDetailsSupplier, Deadline timeout) - throws Exception { + SupplierWithException jobDetailsSupplier) throws Exception { waitUntilCondition( () -> { final JobDetailsInfo jobDetailsInfo = jobDetailsSupplier.get(); @@ -249,39 +214,27 @@ public static void waitForAllTaskRunning( } } return true; - }, - timeout, - "Some tasks are not running until timeout"); + }); } public static void waitForNoTaskRunning( - SupplierWithException jobDetailsSupplier, Deadline timeout) - throws Exception { + SupplierWithException jobDetailsSupplier) throws Exception { waitUntilCondition( () -> { final Map state = jobDetailsSupplier.get().getJobVerticesPerState(); final Integer numRunningTasks = state.get(ExecutionState.RUNNING); return numRunningTasks == null || numRunningTasks.equals(0); - }, - timeout, - "Some tasks are still running until timeout"); + }); } public static void waitUntilJobManagerIsInitialized( SupplierWithException jobStatusSupplier) throws Exception { - waitUntilJobManagerIsInitialized( - jobStatusSupplier, Deadline.fromNow(Duration.of(1, ChronoUnit.MINUTES))); + waitUntilCondition(() -> jobStatusSupplier.get() != JobStatus.INITIALIZING, 20L); } - public static void waitUntilJobManagerIsInitialized( - SupplierWithException jobStatusSupplier, Deadline timeout) + public static void waitForJobStatus(JobClient client, List expectedStatus) throws Exception { - waitUntilCondition(() -> jobStatusSupplier.get() != JobStatus.INITIALIZING, timeout, 20L); - } - - public static void waitForJobStatus( - JobClient client, List expectedStatus, Deadline deadline) throws Exception { waitUntilCondition( () -> { final JobStatus currentStatus = client.getJobStatus().get(); @@ -311,8 +264,7 @@ public static void waitForJobStatus( // Continue waiting for expected status return false; - }, - deadline); + }); } public static void terminateJob(JobClient client) throws Exception { @@ -355,8 +307,49 @@ public static void waitForSubtasksToFinish( return allSubtasks ? vertexStream.allMatch(subtaskPredicate) : vertexStream.anyMatch(subtaskPredicate); - }, - Deadline.fromNow(Duration.of(1, ChronoUnit.MINUTES))); + }); + } + + /** Wait for (at least) the given number of successful checkpoints. */ + public static void waitForCheckpoint(JobID jobID, MiniCluster miniCluster, int numCheckpoints) + throws Exception, FlinkJobNotFoundException { + waitUntilCondition( + () -> { + AccessExecutionGraph graph = miniCluster.getExecutionGraph(jobID).get(); + if (Optional.ofNullable(graph.getCheckpointStatsSnapshot()) + .filter( + st -> + st.getCounts().getNumberOfCompletedCheckpoints() + >= numCheckpoints) + .isPresent()) { + return true; + } else if (graph.getState().isGloballyTerminalState()) { + checkState( + graph.getFailureInfo() != null, + "Job terminated before taking required %s checkpoints: %s", + numCheckpoints, + graph.getState()); + throw graph.getFailureInfo().getException(); + } else { + return false; + } + }); + } + + /** + * @return the path as {@link java.net.URI} to the latest checkpoint. + * @throws FlinkJobNotFoundException if job not found + */ + public static Optional getLatestCompletedCheckpointPath( + JobID jobID, MiniCluster cluster) + throws InterruptedException, ExecutionException, FlinkJobNotFoundException { + return Optional.ofNullable( + cluster.getExecutionGraph(jobID).get().getCheckpointStatsSnapshot()) + .flatMap( + stats -> + Optional.ofNullable( + stats.getHistory().getLatestCompletedCheckpoint())) + .map(CompletedCheckpointStats::getExternalPath); } /** Utility class to read the output of a process stream and forward it into a StringWriter. */ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/MiniClusterResource.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/MiniClusterResource.java index f1971e802edb7..2e24b91d407cf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/MiniClusterResource.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/MiniClusterResource.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.testutils; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.HeartbeatManagerOptions; @@ -43,7 +42,6 @@ import org.slf4j.LoggerFactory; import java.net.URI; -import java.time.Duration; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -128,39 +126,25 @@ public void cancelAllJobs() { private void cancelAllJobs(boolean waitUntilSlotsAreFreed) { try { - final long shutdownTimeout = - miniClusterResourceConfiguration.getShutdownTimeout().toMilliseconds(); - final Deadline jobCancellationDeadline = - Deadline.fromNow(Duration.ofMillis(shutdownTimeout)); final List> jobCancellationFutures = - miniCluster.listJobs() - .get( - jobCancellationDeadline.timeLeft().toMillis(), - TimeUnit.MILLISECONDS) - .stream() + miniCluster.listJobs().get().stream() .filter(status -> !status.getJobState().isGloballyTerminalState()) .map(status -> miniCluster.cancelJob(status.getJobId())) .collect(Collectors.toList()); - FutureUtils.waitForAll(jobCancellationFutures) - .get(jobCancellationDeadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); + FutureUtils.waitForAll(jobCancellationFutures).get(); CommonTestUtils.waitUntilCondition( () -> { final long unfinishedJobs = - miniCluster.listJobs() - .get( - jobCancellationDeadline.timeLeft().toMillis(), - TimeUnit.MILLISECONDS) - .stream() + miniCluster.listJobs().get().stream() .filter( status -> !status.getJobState() .isGloballyTerminalState()) .count(); return unfinishedJobs == 0; - }, - jobCancellationDeadline); + }); if (waitUntilSlotsAreFreed) { CommonTestUtils.waitUntilCondition( @@ -169,8 +153,7 @@ private void cancelAllJobs(boolean waitUntilSlotsAreFreed) { miniCluster.getResourceOverview().get(); return resourceOverview.getNumberRegisteredSlots() == resourceOverview.getNumberFreeSlots(); - }, - jobCancellationDeadline); + }); } } catch (Exception e) { log.warn("Exception while shutting down remaining jobs.", e); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingTaskManagerRuntimeInfo.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingTaskManagerRuntimeInfo.java index ac4023094d271..739716f67a85d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingTaskManagerRuntimeInfo.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingTaskManagerRuntimeInfo.java @@ -38,7 +38,8 @@ public class TestingTaskManagerRuntimeInfo implements TaskManagerRuntimeInfo { public TestingTaskManagerRuntimeInfo() { this( new Configuration(), - System.getProperty("java.io.tmpdir").split(",|" + File.pathSeparator)); + EnvironmentInformation.getTemporaryFileDirectory() + .split(",|" + File.pathSeparator)); } public TestingTaskManagerRuntimeInfo(Configuration configuration) { @@ -62,7 +63,9 @@ public TestingTaskManagerRuntimeInfo(Configuration configuration, String[] tmpDi configuration, tmpDirectories, InetAddress.getLoopbackAddress().getHostAddress(), - new File("tmp_" + UUID.randomUUID())); + new File( + EnvironmentInformation.getTemporaryFileDirectory(), + "tmp_" + UUID.randomUUID())); } public TestingTaskManagerRuntimeInfo( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreTest.java index dc81131fdf6e4..eff53576fc183 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreTest.java @@ -223,6 +223,18 @@ public void testRepeatableCleanup() throws Exception { assertTrue(stateHandle.isDiscarded()); } + @Test + public void testCleanupOfNonExistingState() throws Exception { + final ZooKeeperStateHandleStore testInstance = + new ZooKeeperStateHandleStore<>( + ZOOKEEPER.getClient(), new TestingLongStateHandleHelper()); + + final String pathInZooKeeper = "/testCleanupOfNonExistingState"; + + assertTrue(testInstance.releaseAndTryRemove(pathInZooKeeper)); + assertFalse(testInstance.exists(pathInZooKeeper).isExisting()); + } + @Test public void testRepeatableCleanupWithLockOnNode() throws Exception { final CuratorFramework client = @@ -855,6 +867,46 @@ public void testGetAllAndLockOnConcurrentDelete() throws Exception { Iterables.getOnlyElement(actuallyLockedHandles).f0.retrieveState().getValue()); } + @Test + public void testGetAllAndLockWhileEntryIsMarkedForDeletion() throws Exception { + final TestingLongStateHandleHelper stateHandleProvider = new TestingLongStateHandleHelper(); + final CuratorFramework client = + ZooKeeperUtils.useNamespaceAndEnsurePath( + ZOOKEEPER.getClient(), "/testGetAllAndLockWhileEntryIsMarkedForDeletion"); + + final ZooKeeperStateHandleStore + stateHandleStore = new ZooKeeperStateHandleStore<>(client, stateHandleProvider); + + final String pathInZooKeeperPrefix = "/node"; + + final long stateForDeletion = 42L; + final String handlePathForDeletion = pathInZooKeeperPrefix + "-for-deletion"; + stateHandleStore.addAndLock( + handlePathForDeletion, + new TestingLongStateHandleHelper.LongStateHandle(stateForDeletion)); + // marks the entry for deletion but doesn't delete it, yet + client.delete() + .deletingChildrenIfNeeded() + .forPath(ZooKeeperStateHandleStore.getRootLockPath(handlePathForDeletion)); + + final long stateToKeep = stateForDeletion + 2; + stateHandleStore.addAndLock( + pathInZooKeeperPrefix + "-keep", + new TestingLongStateHandleHelper.LongStateHandle(stateToKeep)); + + final List< + Tuple2< + RetrievableStateHandle< + TestingLongStateHandleHelper.LongStateHandle>, + String>> + actuallyLockedHandles = stateHandleStore.getAllAndLock(); + + assertEquals( + "Only the StateHandle that was expected to be kept should be returned.", + stateToKeep, + Iterables.getOnlyElement(actuallyLockedHandles).f0.retrieveState().getValue()); + } + /** Tests that the state is returned sorted. */ @Test public void testGetAllSortedByName() throws Exception { diff --git a/flink-scala/pom.xml b/flink-scala/pom.xml index 3f8b5a1e63aeb..d333c50ab7a32 100644 --- a/flink-scala/pom.xml +++ b/flink-scala/pom.xml @@ -24,7 +24,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-state-backends/flink-statebackend-changelog/pom.xml b/flink-state-backends/flink-statebackend-changelog/pom.xml index eba8d12572883..c4796a77b992a 100644 --- a/flink-state-backends/flink-statebackend-changelog/pom.xml +++ b/flink-state-backends/flink-statebackend-changelog/pom.xml @@ -27,7 +27,7 @@ under the License. org.apache.flink flink-state-backends - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogStateBackend.java b/flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogStateBackend.java index 863a53382b60e..afcd17aa46c4d 100644 --- a/flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogStateBackend.java +++ b/flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogStateBackend.java @@ -234,7 +234,9 @@ private ChangelogKeyedStateBackend restore( executionConfig, ttlTimeProvider, changelogStorage.createWriter( - operatorIdentifier, keyGroupRange), + operatorIdentifier, + keyGroupRange, + env.getMainMailboxExecutor()), baseState, env.getCheckpointStorageAccess())); diff --git a/flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/PeriodicMaterializationManager.java b/flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/PeriodicMaterializationManager.java index e20c1090699ca..d51d7958cd3f4 100644 --- a/flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/PeriodicMaterializationManager.java +++ b/flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/PeriodicMaterializationManager.java @@ -107,7 +107,8 @@ class PeriodicMaterializationManager implements Closeable { } public void start() { - if (!started) { + // disable periodic materialization when periodicMaterializeDelay is negative + if (!started && periodicMaterializeDelay >= 0) { started = true; diff --git a/flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/ChangelogKeyedStateBackendTest.java b/flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/ChangelogKeyedStateBackendTest.java index 4f9c9104a555f..264fb6a52fa4a 100644 --- a/flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/ChangelogKeyedStateBackendTest.java +++ b/flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/ChangelogKeyedStateBackendTest.java @@ -109,7 +109,7 @@ private ChangelogKeyedStateBackend createChangelog( new ExecutionConfig(), TtlTimeProvider.DEFAULT, new InMemoryStateChangelogStorage() - .createWriter("test", KeyGroupRange.EMPTY_KEY_GROUP_RANGE), + .createWriter("test", KeyGroupRange.EMPTY_KEY_GROUP_RANGE, null), emptyList(), new DummyCheckpointingStorageAccess()); } diff --git a/flink-state-backends/flink-statebackend-heap-spillable/pom.xml b/flink-state-backends/flink-statebackend-heap-spillable/pom.xml index 3a2e7248ba28d..7669fd37cf4da 100644 --- a/flink-state-backends/flink-statebackend-heap-spillable/pom.xml +++ b/flink-state-backends/flink-statebackend-heap-spillable/pom.xml @@ -27,7 +27,7 @@ under the License. org.apache.flink flink-state-backends - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-state-backends/flink-statebackend-rocksdb/pom.xml b/flink-state-backends/flink-statebackend-rocksdb/pom.xml index 07f8ba451d9cd..855ff764d8878 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/pom.xml +++ b/flink-state-backends/flink-statebackend-rocksdb/pom.xml @@ -27,7 +27,7 @@ under the License. org.apache.flink flink-state-backends - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/PredefinedOptions.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/PredefinedOptions.java index 25e5fa04d5908..d69b54314f0ac 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/PredefinedOptions.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/PredefinedOptions.java @@ -22,7 +22,6 @@ import org.apache.flink.configuration.MemorySize; import org.rocksdb.CompactionStyle; -import org.rocksdb.InfoLogLevel; import javax.annotation.Nullable; @@ -54,15 +53,9 @@ public enum PredefinedOptions { *

Note: Because Flink does not rely on RocksDB data on disk for recovery, there is no need * to sync data to stable storage. * - *

The following options are set: - * - *

    - *
  • setInfoLogLevel(InfoLogLevel.HEADER_LEVEL) - *
+ *

There are no specified options here. */ - DEFAULT( - Collections.singletonMap( - RocksDBConfigurableOptions.LOG_LEVEL, InfoLogLevel.HEADER_LEVEL)), + DEFAULT(Collections.emptyMap()), /** * Pre-defined options for regular spinning hard disks. @@ -78,7 +71,6 @@ public enum PredefinedOptions { *

  • setMaxBackgroundJobs(4) *
  • setDisableDataSync(true) *
  • setMaxOpenFiles(-1) - *
  • setInfoLogLevel(InfoLogLevel.HEADER_LEVEL) * * *

    Note: Because Flink does not rely on RocksDB data on disk for recovery, there is no need @@ -91,7 +83,6 @@ public enum PredefinedOptions { { put(RocksDBConfigurableOptions.MAX_BACKGROUND_THREADS, 4); put(RocksDBConfigurableOptions.MAX_OPEN_FILES, -1); - put(RocksDBConfigurableOptions.LOG_LEVEL, InfoLogLevel.HEADER_LEVEL); put(RocksDBConfigurableOptions.COMPACTION_STYLE, CompactionStyle.LEVEL); put(RocksDBConfigurableOptions.USE_DYNAMIC_LEVEL_SIZE, true); } @@ -116,7 +107,6 @@ public enum PredefinedOptions { *

  • setMinWriteBufferNumberToMerge(3) *
  • setMaxWriteBufferNumber(4) *
  • setMaxOpenFiles(-1) - *
  • setInfoLogLevel(InfoLogLevel.HEADER_LEVEL) *
  • BlockBasedTableConfig.setBlockCacheSize(256 MBytes) *
  • BlockBasedTableConfig.setBlockSize(128 KBytes) * @@ -131,7 +121,6 @@ public enum PredefinedOptions { { put(RocksDBConfigurableOptions.MAX_BACKGROUND_THREADS, 4); put(RocksDBConfigurableOptions.MAX_OPEN_FILES, -1); - put(RocksDBConfigurableOptions.LOG_LEVEL, InfoLogLevel.HEADER_LEVEL); put(RocksDBConfigurableOptions.COMPACTION_STYLE, CompactionStyle.LEVEL); put(RocksDBConfigurableOptions.USE_DYNAMIC_LEVEL_SIZE, true); put( @@ -159,7 +148,6 @@ public enum PredefinedOptions { *
  • setMaxBackgroundJobs(4) *
  • setDisableDataSync(true) *
  • setMaxOpenFiles(-1) - *
  • setInfoLogLevel(InfoLogLevel.HEADER_LEVEL) * * *

    Note: Because Flink does not rely on RocksDB data on disk for recovery, there is no need @@ -172,7 +160,6 @@ public enum PredefinedOptions { { put(RocksDBConfigurableOptions.MAX_BACKGROUND_THREADS, 4); put(RocksDBConfigurableOptions.MAX_OPEN_FILES, -1); - put(RocksDBConfigurableOptions.LOG_LEVEL, InfoLogLevel.HEADER_LEVEL); } }); diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java index a1d93df798a8e..8a4af6b7b89d4 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java @@ -102,6 +102,9 @@ public void testDefaultDbLogDir() throws Exception { // set the environment variable 'log.file' with the Flink log file location System.setProperty("log.file", logFile.getPath()); try (RocksDBResourceContainer container = backend.createOptionsAndResourceContainer()) { + assertEquals( + RocksDBConfigurableOptions.LOG_LEVEL.defaultValue(), + container.getDbOptions().infoLogLevel()); assertEquals(logFile.getParent(), container.getDbOptions().dbLogDir()); } finally { logFile.delete(); diff --git a/flink-state-backends/pom.xml b/flink-state-backends/pom.xml index 805bd6252d72b..42a4adf25ae31 100644 --- a/flink-state-backends/pom.xml +++ b/flink-state-backends/pom.xml @@ -27,7 +27,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-streaming-java/pom.xml b/flink-streaming-java/pom.xml index bc87e1d0e9d21..3dd3f10025845 100644 --- a/flink-streaming-java/pom.xml +++ b/flink-streaming-java/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/ExternallyInducedSource.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/ExternallyInducedSource.java index eafd3074d1554..537a56f63df24 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/ExternallyInducedSource.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/ExternallyInducedSource.java @@ -23,8 +23,8 @@ import org.apache.flink.util.FlinkException; /** - * Sources that implement this interface do not trigger checkpoints when receiving a trigger message - * from the checkpoint coordinator, but when their input data/events indicate that a checkpoint + * Sources that implement this interface delay checkpoints when receiving a trigger message from the + * checkpoint coordinator to the point when their input data/events indicate that a checkpoint * should be triggered. * *

    Since sources cannot simply create a new checkpoint on their own, this mechanism always goes diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CustomSinkOperatorUidHashes.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CustomSinkOperatorUidHashes.java new file mode 100644 index 0000000000000..ae4a60eb6e435 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CustomSinkOperatorUidHashes.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.api.datastream; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.PublicEvolving; + +import javax.annotation.Nullable; + +/** + * This class is responsible to hold operator Uid hashes from the common operators of the sink. With + * this, users can recover a sink snapshot that did not bind uids to the operator before changing + * the topology. + */ +@PublicEvolving +public class CustomSinkOperatorUidHashes { + + /** Default instance providing no custom sink operator hashes. */ + public static final CustomSinkOperatorUidHashes DEFAULT = + CustomSinkOperatorUidHashes.builder().build(); + + @Nullable private final String writerUidHash; + @Nullable private final String committerUidHash; + @Nullable private final String globalCommitterUidHash; + + private CustomSinkOperatorUidHashes( + @Nullable String writerUidHash, + @Nullable String committerUidHash, + @Nullable String globalCommitterUidHash) { + this.writerUidHash = writerUidHash; + this.committerUidHash = committerUidHash; + this.globalCommitterUidHash = globalCommitterUidHash; + } + + /** + * Creates a builder to construct {@link CustomSinkOperatorUidHashes}. + * + * @return {@link SinkOperatorUidHashesBuilder} + */ + public static SinkOperatorUidHashesBuilder builder() { + return new SinkOperatorUidHashesBuilder(); + } + + @Internal + @Nullable + public String getWriterUidHash() { + return writerUidHash; + } + + @Internal + @Nullable + public String getCommitterUidHash() { + return committerUidHash; + } + + @Internal + @Nullable + public String getGlobalCommitterUidHash() { + return globalCommitterUidHash; + } + + /** Builder to construct {@link CustomSinkOperatorUidHashes}. */ + @PublicEvolving + public static class SinkOperatorUidHashesBuilder { + + @Nullable String writerUidHash = null; + @Nullable String committerUidHash = null; + @Nullable String globalCommitterUidHash = null; + + /** + * Sets the uid hash of the writer operator used to recover state. + * + * @param writerUidHash uid hash denoting writer operator + * @return {@link SinkOperatorUidHashesBuilder} + */ + public SinkOperatorUidHashesBuilder setWriterUidHash(String writerUidHash) { + this.writerUidHash = writerUidHash; + return this; + } + + /** + * Sets the uid hash of the committer operator used to recover state. + * + * @param committerUidHash uid hash denoting the committer operator + * @return {@link SinkOperatorUidHashesBuilder} + */ + public SinkOperatorUidHashesBuilder setCommitterUidHash(String committerUidHash) { + this.committerUidHash = committerUidHash; + return this; + } + + /** + * Sets the uid hash of the global committer operator used to recover state. + * + * @param globalCommitterUidHash uid hash denoting the global committer operator + * @return {@link SinkOperatorUidHashesBuilder} + */ + public SinkOperatorUidHashesBuilder setGlobalCommitterUidHash( + String globalCommitterUidHash) { + this.globalCommitterUidHash = globalCommitterUidHash; + return this; + } + + /** + * Constructs the {@link CustomSinkOperatorUidHashes} with the given uid hashes. + * + * @return {@link CustomSinkOperatorUidHashes} + */ + public CustomSinkOperatorUidHashes build() { + return new CustomSinkOperatorUidHashes( + writerUidHash, committerUidHash, globalCommitterUidHash); + } + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java index 3382b01804d71..b0c3d64966579 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java @@ -1250,10 +1250,27 @@ public DataStreamSink addSink(SinkFunction sinkFunction) { */ @PublicEvolving public DataStreamSink sinkTo(org.apache.flink.api.connector.sink.Sink sink) { + return this.sinkTo(sink, CustomSinkOperatorUidHashes.DEFAULT); + } + + /** + * Adds the given {@link Sink} to this DataStream. Only streams with sinks added will be + * executed once the {@link StreamExecutionEnvironment#execute()} method is called. + * + *

    This method is intended to be used only to recover a snapshot where no uids have been set + * before taking the snapshot. + * + * @param sink The user defined sink. + * @return The closed DataStream. + */ + @PublicEvolving + public DataStreamSink sinkTo( + org.apache.flink.api.connector.sink.Sink sink, + CustomSinkOperatorUidHashes customSinkOperatorUidHashes) { // read the output type of the input Transform to coax out errors about MissingTypeInfo transformation.getOutputType(); - return DataStreamSink.forSinkV1(this, sink); + return DataStreamSink.forSinkV1(this, sink, customSinkOperatorUidHashes); } /** @@ -1265,10 +1282,27 @@ public DataStreamSink sinkTo(org.apache.flink.api.connector.sink.Sink sinkTo(Sink sink) { + return this.sinkTo(sink, CustomSinkOperatorUidHashes.DEFAULT); + } + + /** + * Adds the given {@link Sink} to this DataStream. Only streams with sinks added will be + * executed once the {@link StreamExecutionEnvironment#execute()} method is called. + * + *

    This method is intended to be used only to recover a snapshot where no uids have been set + * before taking the snapshot. + * + * @param customSinkOperatorUidHashes operator hashes to support state binding + * @param sink The user defined sink. + * @return The closed DataStream. + */ + @PublicEvolving + public DataStreamSink sinkTo( + Sink sink, CustomSinkOperatorUidHashes customSinkOperatorUidHashes) { // read the output type of the input Transform to coax out errors about MissingTypeInfo transformation.getOutputType(); - return DataStreamSink.forSink(this, sink); + return DataStreamSink.forSink(this, sink, customSinkOperatorUidHashes); } /** diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java index f93c2c09ebf11..f9d60e722538f 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java @@ -65,7 +65,10 @@ static DataStreamSink forSinkFunction( } @Internal - public static DataStreamSink forSink(DataStream inputStream, Sink sink) { + public static DataStreamSink forSink( + DataStream inputStream, + Sink sink, + CustomSinkOperatorUidHashes customSinkOperatorUidHashes) { final StreamExecutionEnvironment executionEnvironment = inputStream.getExecutionEnvironment(); SinkTransformation transformation = @@ -74,15 +77,18 @@ public static DataStreamSink forSink(DataStream inputStream, Sink s sink, inputStream.getType(), "Sink", - executionEnvironment.getParallelism()); + executionEnvironment.getParallelism(), + customSinkOperatorUidHashes); executionEnvironment.addOperator(transformation); return new DataStreamSink<>(transformation); } @Internal public static DataStreamSink forSinkV1( - DataStream inputStream, org.apache.flink.api.connector.sink.Sink sink) { - return forSink(inputStream, SinkV1Adapter.wrap(sink)); + DataStream inputStream, + org.apache.flink.api.connector.sink.Sink sink, + CustomSinkOperatorUidHashes customSinkOperatorUidHashes) { + return forSink(inputStream, SinkV1Adapter.wrap(sink), customSinkOperatorUidHashes); } /** Returns the transformation that contains the actual sink operator of this sink. */ diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java index f1f34baf1e25a..8b5d2ab73fe5f 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java @@ -142,6 +142,7 @@ private void restoreInProgressFile(final BucketState state) throws IOE bucketId, inProgressFileRecoverable, state.getInProgressFileCreationTime()); + inProgressFileRecoverablesPerCheckpoint.put(Long.MIN_VALUE, inProgressFileRecoverable); } else { // if the writer does not support resume, then we close the // in-progress part and commit it, as done in the case of pending files. diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImpl.java index a9977d79bd80d..b2cf4c6b94aee 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImpl.java @@ -279,8 +279,8 @@ private void onProcessingTime(long time) throws Exception { InternalTimer timer; while ((timer = processingTimeTimersQueue.peek()) != null && timer.getTimestamp() <= time) { - processingTimeTimersQueue.poll(); keyContext.setCurrentKey(timer.getKey()); + processingTimeTimersQueue.poll(); triggerTarget.onProcessingTime(timer); } @@ -297,8 +297,8 @@ public void advanceWatermark(long time) throws Exception { InternalTimer timer; while ((timer = eventTimeTimersQueue.peek()) != null && timer.getTimestamp() <= time) { - eventTimeTimersQueue.poll(); keyContext.setCurrentKey(timer.getKey()); + eventTimeTimersQueue.poll(); triggerTarget.onEventTime(timer); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java index aba208549aa35..53621bd1b6e0b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java @@ -67,6 +67,7 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.util.ArrayList; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -154,6 +155,8 @@ public class SourceOperator extends AbstractStr private final SourceOperatorAvailabilityHelper availabilityHelper = new SourceOperatorAvailabilityHelper(); + private final List outputPendingSplits = new ArrayList<>(); + private enum OperatingMode { READING, WAITING_FOR_ALIGNMENT, @@ -170,7 +173,7 @@ private enum OperatingMode { private CompletableFuture waitingForAlignmentFuture = CompletableFuture.completedFuture(null); - private @Nullable LatencyMarkerEmitter latencyMarerEmitter; + private @Nullable LatencyMarkerEmitter latencyMarkerEmitter; public SourceOperator( FunctionWithException, Exception> @@ -334,8 +337,8 @@ private void stopInternalServices() { if (eventTimeLogic != null) { eventTimeLogic.stopPeriodicWatermarkEmits(); } - if (latencyMarerEmitter != null) { - latencyMarerEmitter.close(); + if (latencyMarkerEmitter != null) { + latencyMarkerEmitter.close(); } } @@ -396,11 +399,7 @@ private DataInputStatus emitNextNotReading(DataOutput output) throws Except watermarkAlignmentParams.getUpdateInterval(), watermarkAlignmentParams.getUpdateInterval()); } - currentMainOutput = - eventTimeLogic.createMainOutput(output, this::onWatermarkEmitted); - initializeLatencyMarkerEmitter(output); - lastInvokedOutput = output; - this.operatingMode = OperatingMode.READING; + initializeMainOutput(output); return convertToInternalStatus(sourceReader.pollNext(currentMainOutput)); case SOURCE_STOPPED: this.operatingMode = OperatingMode.DATA_FINISHED; @@ -423,6 +422,15 @@ private DataInputStatus emitNextNotReading(DataOutput output) throws Except } } + private void initializeMainOutput(DataOutput output) { + currentMainOutput = eventTimeLogic.createMainOutput(output, this::onWatermarkEmitted); + initializeLatencyMarkerEmitter(output); + lastInvokedOutput = output; + // Create per-split output for pending splits added before main output is initialized + createOutputForSplits(outputPendingSplits); + this.operatingMode = OperatingMode.READING; + } + private void initializeLatencyMarkerEmitter(DataOutput output) { long latencyTrackingInterval = getExecutionConfig().isLatencyTrackingConfigured() @@ -433,7 +441,7 @@ private void initializeLatencyMarkerEmitter(DataOutput output) { .getConfiguration() .getLong(MetricOptions.LATENCY_INTERVAL); if (latencyTrackingInterval > 0) { - latencyMarerEmitter = + latencyMarkerEmitter = new LatencyMarkerEmitter<>( getProcessingTimeService(), output::emitLatencyMarker, @@ -515,11 +523,7 @@ public void handleOperatorEvent(OperatorEvent event) { updateMaxDesiredWatermark((WatermarkAlignmentEvent) event); checkWatermarkAlignment(); } else if (event instanceof AddSplitEvent) { - try { - sourceReader.addSplits(((AddSplitEvent) event).splits(splitSerializer)); - } catch (IOException e) { - throw new FlinkRuntimeException("Failed to deserialize the splits.", e); - } + handleAddSplitsEvent(((AddSplitEvent) event)); } else if (event instanceof SourceEventWrapper) { sourceReader.handleSourceEvents(((SourceEventWrapper) event).getSourceEvent()); } else if (event instanceof NoMoreSplitsEvent) { @@ -529,6 +533,30 @@ public void handleOperatorEvent(OperatorEvent event) { } } + private void handleAddSplitsEvent(AddSplitEvent event) { + try { + List newSplits = event.splits(splitSerializer); + if (operatingMode == OperatingMode.OUTPUT_NOT_INITIALIZED) { + // For splits arrived before the main output is initialized, store them into the + // pending list. Outputs of these splits will be created once the main output is + // ready. + outputPendingSplits.addAll(newSplits); + } else { + // Create output directly for new splits if the main output is already initialized. + createOutputForSplits(newSplits); + } + sourceReader.addSplits(newSplits); + } catch (IOException e) { + throw new FlinkRuntimeException("Failed to deserialize the splits.", e); + } + } + + private void createOutputForSplits(List newSplits) { + for (SplitT split : newSplits) { + currentMainOutput.createOutputForSplit(split.splitId()); + } + } + private void updateMaxDesiredWatermark(WatermarkAlignmentEvent event) { currentMaxDesiredWatermark = event.getMaxWatermark(); sourceMetricGroup.updateMaxDesiredWatermark(currentMaxDesiredWatermark); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java index 967f3642c57f8..bb11c260e95fb 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java @@ -23,6 +23,7 @@ Licensed to the Apache Software Foundation (ASF) under one import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.api.connector.sink2.SinkWriter; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.streaming.api.datastream.CustomSinkOperatorUidHashes; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.operators.ChainingStrategy; @@ -47,6 +48,7 @@ public class SinkTransformation extends PhysicalTransformation< private final DataStream inputStream; private final Sink sink; private final Transformation input; + private final CustomSinkOperatorUidHashes customSinkOperatorUidHashes; private ChainingStrategy chainingStrategy; @@ -55,11 +57,13 @@ public SinkTransformation( Sink sink, TypeInformation outputType, String name, - int parallelism) { + int parallelism, + CustomSinkOperatorUidHashes customSinkOperatorUidHashes) { super(name, outputType, parallelism); this.inputStream = checkNotNull(inputStream); this.sink = checkNotNull(sink); this.input = inputStream.getTransformation(); + this.customSinkOperatorUidHashes = checkNotNull(customSinkOperatorUidHashes); } @Override @@ -92,4 +96,8 @@ public DataStream getInputStream() { public Sink getSink() { return sink; } + + public CustomSinkOperatorUidHashes getSinkOperatorsUidHashes() { + return customSinkOperatorUidHashes; + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/windows/TimeWindow.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/windows/TimeWindow.java index 3346b3978590b..93c1bf16f6bf1 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/windows/TimeWindow.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/windows/TimeWindow.java @@ -262,6 +262,12 @@ public int compare(TimeWindow o1, TimeWindow o2) { * @return window start */ public static long getWindowStartWithOffset(long timestamp, long offset, long windowSize) { - return timestamp - (timestamp - offset + windowSize) % windowSize; + final long remainder = (timestamp - offset) % windowSize; + // handle both positive and negative cases + if (remainder < 0) { + return timestamp - (remainder + windowSize); + } else { + return timestamp - remainder; + } } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskExternallyInducedSourceInput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskExternallyInducedSourceInput.java index fff008cfa69d8..ca0462eec19d9 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskExternallyInducedSourceInput.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskExternallyInducedSourceInput.java @@ -21,12 +21,14 @@ import org.apache.flink.api.connector.source.ExternallyInducedSourceReader; import org.apache.flink.streaming.api.operators.SourceOperator; +import java.util.concurrent.CompletableFuture; import java.util.function.Consumer; /** A subclass of {@link StreamTaskSourceInput} for {@link ExternallyInducedSourceReader}. */ public class StreamTaskExternallyInducedSourceInput extends StreamTaskSourceInput { private final Consumer checkpointTriggeringHook; private final ExternallyInducedSourceReader sourceReader; + private CompletableFuture blockFuture; @SuppressWarnings("unchecked") public StreamTaskExternallyInducedSourceInput( @@ -39,12 +41,34 @@ public StreamTaskExternallyInducedSourceInput( this.sourceReader = (ExternallyInducedSourceReader) operator.getSourceReader(); } + public void blockUntil(CompletableFuture blockFuture) { + this.blockFuture = blockFuture; + // assume that the future is completed in mailbox thread + blockFuture.whenComplete((v, e) -> unblock()); + } + + private void unblock() { + this.blockFuture = null; + } + @Override public DataInputStatus emitNext(DataOutput output) throws Exception { + if (blockFuture != null) { + return DataInputStatus.NOTHING_AVAILABLE; + } + DataInputStatus status = super.emitNext(output); if (status == DataInputStatus.NOTHING_AVAILABLE) { sourceReader.shouldTriggerCheckpoint().ifPresent(checkpointTriggeringHook); } return status; } + + @Override + public CompletableFuture getAvailableFuture() { + if (blockFuture != null) { + return blockFuture; + } + return super.getAvailableFuture(); + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/TimestampsAndWatermarksOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/TimestampsAndWatermarksOperator.java index a10c92e1e5ff2..570cb6b90b3e6 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/TimestampsAndWatermarksOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/TimestampsAndWatermarksOperator.java @@ -126,6 +126,10 @@ public void processWatermark(org.apache.flink.streaming.api.watermark.Watermark } } + /** Override the base implementation to completely ignore statuses propagated from upstream. */ + @Override + public void processWatermarkStatus(WatermarkStatus watermarkStatus) throws Exception {} + @Override public void finish() throws Exception { super.finish(); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTask.java index efc594b6940db..3d2805599fbb6 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTask.java @@ -24,14 +24,12 @@ import org.apache.flink.metrics.Counter; import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointOptions; -import org.apache.flink.runtime.checkpoint.CheckpointType; import org.apache.flink.runtime.checkpoint.SavepointType; import org.apache.flink.runtime.checkpoint.SnapshotType; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.api.StopMode; import org.apache.flink.runtime.metrics.MetricNames; import org.apache.flink.runtime.metrics.groups.InternalSourceReaderMetricGroup; -import org.apache.flink.runtime.state.CheckpointStorageLocationReference; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.SourceOperator; import org.apache.flink.streaming.api.watermark.Watermark; @@ -48,7 +46,12 @@ import javax.annotation.Nullable; +import java.util.SortedMap; +import java.util.SortedSet; +import java.util.TreeMap; +import java.util.TreeSet; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Future; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -57,7 +60,25 @@ public class SourceOperatorStreamTask extends StreamTask> { private AsyncDataOutputToOutput output; - private boolean isExternallyInducedSource; + /** + * Contains information about all checkpoints where RPC from checkpoint coordinator arrives + * before the source reader triggers it. (Common case) + */ + private SortedMap untriggeredCheckpoints = new TreeMap<>(); + /** + * Contains the checkpoints that are triggered by the source but the RPC from checkpoint + * coordinator has yet to arrive. This may happen if the barrier is inserted as an event into + * the data plane by the source coordinator and the (distributed) source reader reads that event + * before receiving Flink's checkpoint RPC. (Rare case) + */ + private SortedSet triggeredCheckpoints = new TreeSet<>(); + /** + * Blocks input until the RPC call has been received that corresponds to the triggered + * checkpoint. This future must only be accessed and completed in the mailbox thread. + */ + private CompletableFuture waitForRPC = FutureUtils.completedVoidFuture(); + /** Only set for externally induced sources. See also {@link #isExternallyInducedSource()}. */ + private StreamTaskExternallyInducedSourceInput externallyInducedSourceInput; public SourceOperatorStreamTask(Environment env) throws Exception { super(env); @@ -79,14 +100,14 @@ public void init() throws Exception { if (operatorChain.isTaskDeployedAsFinished()) { input = new StreamTaskFinishedOnRestoreSourceInput<>(sourceOperator, 0, 0); } else if (sourceReader instanceof ExternallyInducedSourceReader) { - isExternallyInducedSource = true; - - input = + externallyInducedSourceInput = new StreamTaskExternallyInducedSourceInput<>( sourceOperator, this::triggerCheckpointForExternallyInducedSource, 0, 0); + + input = externallyInducedSourceInput; } else { input = new StreamTaskSourceInput<>(sourceOperator, 0, 0); } @@ -112,20 +133,53 @@ public void init() throws Exception { @Override public CompletableFuture triggerCheckpointAsync( CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions) { - if (!isExternallyInducedSource) { - if (isSynchronous(checkpointOptions.getCheckpointType())) { - return triggerStopWithSavepointAsync(checkpointMetaData, checkpointOptions); - } else { - return super.triggerCheckpointAsync(checkpointMetaData, checkpointOptions); - } - } else if (checkpointOptions.getCheckpointType().equals(CheckpointType.FULL_CHECKPOINT)) { - // see FLINK-25256 - throw new IllegalStateException( - "Using externally induced sources, we can not enforce taking a full checkpoint." - + "If you are restoring from a snapshot in NO_CLAIM mode, please use" - + " either CLAIM or LEGACY mode."); + if (!isExternallyInducedSource()) { + return triggerCheckpointNowAsync(checkpointMetaData, checkpointOptions); + } + CompletableFuture triggerFuture = new CompletableFuture<>(); + // immediately move RPC to mailbox so we don't need to synchronize fields + mainMailboxExecutor.execute( + () -> + triggerCheckpointOnExternallyInducedSource( + checkpointMetaData, checkpointOptions, triggerFuture), + "SourceOperatorStreamTask#triggerCheckpointAsync(%s, %s)", + checkpointMetaData, + checkpointOptions); + return triggerFuture; + } + + private boolean isExternallyInducedSource() { + return externallyInducedSourceInput != null; + } + + private void triggerCheckpointOnExternallyInducedSource( + CheckpointMetaData checkpointMetaData, + CheckpointOptions checkpointOptions, + CompletableFuture triggerFuture) { + assert (mailboxProcessor.isMailboxThread()); + if (!triggeredCheckpoints.remove(checkpointMetaData.getCheckpointId())) { + // common case: RPC is received before source reader triggers checkpoint + // store metadata and options for later + untriggeredCheckpoints.put( + checkpointMetaData.getCheckpointId(), + new UntriggeredCheckpoint(checkpointMetaData, checkpointOptions)); + triggerFuture.complete(isRunning()); + } else { + // trigger already received (rare case) + FutureUtils.forward( + triggerCheckpointNowAsync(checkpointMetaData, checkpointOptions), + triggerFuture); + + cleanupOldCheckpoints(checkpointMetaData.getCheckpointId()); + } + } + + private CompletableFuture triggerCheckpointNowAsync( + CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions) { + if (isSynchronous(checkpointOptions.getCheckpointType())) { + return triggerStopWithSavepointAsync(checkpointMetaData, checkpointOptions); } else { - return CompletableFuture.completedFuture(isRunning()); + return super.triggerCheckpointAsync(checkpointMetaData, checkpointOptions); } } @@ -159,22 +213,76 @@ protected void advanceToEndOfEventTime() { output.emitWatermark(Watermark.MAX_WATERMARK); } + @Override + protected void declineCheckpoint(long checkpointId) { + cleanupCheckpoint(checkpointId); + super.declineCheckpoint(checkpointId); + } + + @Override + public Future notifyCheckpointAbortAsync( + long checkpointId, long latestCompletedCheckpointId) { + mainMailboxExecutor.execute( + () -> cleanupCheckpoint(checkpointId), "Cleanup checkpoint %d", checkpointId); + return super.notifyCheckpointAbortAsync(checkpointId, latestCompletedCheckpointId); + } + + @Override + public Future notifyCheckpointSubsumedAsync(long checkpointId) { + mainMailboxExecutor.execute( + () -> cleanupCheckpoint(checkpointId), "Cleanup checkpoint %d", checkpointId); + return super.notifyCheckpointSubsumedAsync(checkpointId); + } + // -------------------------- private void triggerCheckpointForExternallyInducedSource(long checkpointId) { - final CheckpointOptions checkpointOptions = - CheckpointOptions.forConfig( - CheckpointType.CHECKPOINT, - CheckpointStorageLocationReference.getDefault(), - configuration.isExactlyOnceCheckpointMode(), - configuration.isUnalignedCheckpointsEnabled(), - configuration.getAlignedCheckpointTimeout().toMillis()); - final long timestamp = System.currentTimeMillis(); + UntriggeredCheckpoint untriggeredCheckpoint = untriggeredCheckpoints.remove(checkpointId); + if (untriggeredCheckpoint != null) { + // common case: RPC before external sources induces it + triggerCheckpointNowAsync( + untriggeredCheckpoint.getMetadata(), + untriggeredCheckpoint.getCheckpointOptions()); + cleanupOldCheckpoints(checkpointId); + } else { + // rare case: external source induced first + triggeredCheckpoints.add(checkpointId); + if (waitForRPC.isDone()) { + waitForRPC = new CompletableFuture<>(); + externallyInducedSourceInput.blockUntil(waitForRPC); + } + } + } + + /** + * Cleanup any orphaned checkpoint before the given currently triggered checkpoint. These + * checkpoint may occur when the checkpoint is cancelled but the RPC is lost. Note, to be safe, + * checkpoint X is only removed when both RPC and trigger for a checkpoint Y>X is received. + */ + private void cleanupOldCheckpoints(long checkpointId) { + assert (mailboxProcessor.isMailboxThread()); + triggeredCheckpoints.headSet(checkpointId).clear(); + untriggeredCheckpoints.headMap(checkpointId).clear(); + + maybeResumeProcessing(); + } + + /** Resumes processing if it was blocked before or else is a no-op. */ + private void maybeResumeProcessing() { + assert (mailboxProcessor.isMailboxThread()); - final CheckpointMetaData checkpointMetaData = - new CheckpointMetaData(checkpointId, timestamp, timestamp); + if (triggeredCheckpoints.isEmpty()) { + waitForRPC.complete(null); + } + } + + /** Remove temporary data about a canceled checkpoint. */ + private void cleanupCheckpoint(long checkpointId) { + assert (mailboxProcessor.isMailboxThread()); + triggeredCheckpoints.remove(checkpointId); + untriggeredCheckpoints.remove(checkpointId); - super.triggerCheckpointAsync(checkpointMetaData, checkpointOptions); + maybeResumeProcessing(); } // --------------------------- @@ -225,4 +333,23 @@ public void emitWatermarkStatus(WatermarkStatus watermarkStatus) throws Exceptio output.emitWatermarkStatus(watermarkStatus); } } + + private static class UntriggeredCheckpoint { + private final CheckpointMetaData metadata; + private final CheckpointOptions checkpointOptions; + + private UntriggeredCheckpoint( + CheckpointMetaData metadata, CheckpointOptions checkpointOptions) { + this.metadata = metadata; + this.checkpointOptions = checkpointOptions; + } + + public CheckpointMetaData getMetadata() { + return metadata; + } + + public CheckpointOptions getCheckpointOptions() { + return checkpointOptions; + } + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/SinkTransformationTranslator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/SinkTransformationTranslator.java index 1bdcc505a83d1..0dd608755cfc9 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/SinkTransformationTranslator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/SinkTransformationTranslator.java @@ -31,6 +31,7 @@ import org.apache.flink.streaming.api.connector.sink2.WithPostCommitTopology; import org.apache.flink.streaming.api.connector.sink2.WithPreCommitTopology; import org.apache.flink.streaming.api.connector.sink2.WithPreWriteTopology; +import org.apache.flink.streaming.api.datastream.CustomSinkOperatorUidHashes; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.graph.TransformationTranslator; @@ -257,7 +258,10 @@ private R adjustTransformations( List> expandedTransformations = transformations.subList(numTransformsBefore, transformations.size()); + final CustomSinkOperatorUidHashes operatorsUidHashes = + transformation.getSinkOperatorsUidHashes(); for (Transformation subTransformation : expandedTransformations) { + String subUid = subTransformation.getUid(); if (isExpandedTopology && subUid != null && !subUid.isEmpty()) { checkState( @@ -268,6 +272,18 @@ private R adjustTransformations( + " has set uid for some operators."); } + // Set the operator uid hashes to support stateful upgrades without prior uids + setOperatorUidHashIfPossible( + subTransformation, WRITER_NAME, operatorsUidHashes.getWriterUidHash()); + setOperatorUidHashIfPossible( + subTransformation, + COMMITTER_NAME, + operatorsUidHashes.getCommitterUidHash()); + setOperatorUidHashIfPossible( + subTransformation, + StandardSinkTopologies.GLOBAL_COMMITTER_TRANSFORMATION_NAME, + operatorsUidHashes.getGlobalCommitterUidHash()); + concatUid( subTransformation, Transformation::getUid, @@ -323,15 +339,26 @@ private R adjustTransformations( return result; } + private void setOperatorUidHashIfPossible( + Transformation transformation, + String writerName, + @Nullable String operatorUidHash) { + if (operatorUidHash == null || !transformation.getName().equals(writerName)) { + return; + } + transformation.setUidHash(operatorUidHash); + } + private void concatUid( Transformation subTransformation, Function, String> getter, BiConsumer, String> setter, @Nullable String transformationName) { if (transformationName != null && getter.apply(transformation) != null) { - // Use the same uid pattern than for Sink V1 + // Use the same uid pattern than for Sink V1. We deliberately decided to use the uid + // pattern of Flink 1.13 because 1.14 did not have a dedicated committer operator. if (transformationName.equals(COMMITTER_NAME)) { - final String committerFormat = "Sink %s Committer"; + final String committerFormat = "Sink Committer: %s"; setter.accept( subTransformation, String.format(committerFormat, getter.apply(transformation))); @@ -343,7 +370,7 @@ private void concatUid( return; } - // Use the same uid pattern than for Sink V1 + // Use the same uid pattern than for Sink V1 in Flink 1.14. if (transformationName.equals( StandardSinkTopologies.GLOBAL_COMMITTER_TRANSFORMATION_NAME)) { final String committerFormat = "Sink %s Global Committer"; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java index 9fc09cdb110f0..245e895878b41 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java @@ -117,6 +117,33 @@ public void shouldNotCallCleanupWithoutInProgressPartFiles() throws Exception { assertThat(recoverableWriter, hasCalledDiscard(0)); // we have no in-progress file. } + @Test + public void shouldCleanupOutdatedResumablesAfterResumed() throws Exception { + final File outDir = TEMP_FOLDER.newFolder(); + final Path path = new Path(outDir.toURI()); + + final TestRecoverableWriter recoverableWriter = getRecoverableWriter(path); + final Bucket bucketUnderTest = + createBucket(recoverableWriter, path, 0, 0, OutputFileConfig.builder().build()); + + bucketUnderTest.write("test-element", 0L); + final BucketState state0 = bucketUnderTest.onReceptionOfCheckpoint(0L); + assertThat(state0, hasActiveInProgressFile()); + bucketUnderTest.onSuccessfulCompletionOfCheckpoint(0L); + assertThat(recoverableWriter, hasCalledDiscard(0)); + + final File newOutDir = TEMP_FOLDER.newFolder(); + final Path newPath = new Path(newOutDir.toURI()); + final TestRecoverableWriter newRecoverableWriter = getRecoverableWriter(newPath); + final Bucket bucketAfterResume = + restoreBucket( + newRecoverableWriter, 0, 0, state0, OutputFileConfig.builder().build()); + final BucketState state1 = bucketAfterResume.onReceptionOfCheckpoint(1L); + assertThat(state1, hasActiveInProgressFile()); + bucketAfterResume.onSuccessfulCompletionOfCheckpoint(1L); + assertThat(newRecoverableWriter, hasCalledDiscard(1)); + } + // --------------------------- Checking Restore --------------------------- @Test diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkTransformationTranslatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkTransformationTranslatorTest.java index 44189e061980e..e2086ec63ac38 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkTransformationTranslatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkTransformationTranslatorTest.java @@ -23,6 +23,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ExecutionOptions; import org.apache.flink.core.io.SimpleVersionedSerializerTypeSerializerProxy; +import org.apache.flink.streaming.api.datastream.CustomSinkOperatorUidHashes; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -47,6 +48,7 @@ import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.CoreMatchers.not; import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertEquals; /** Tests for {@link org.apache.flink.streaming.api.transformations.SinkTransformation}. */ @RunWith(Parameterized.class) @@ -263,6 +265,57 @@ public void disableOperatorChain() { is(ChainingStrategy.NEVER)); } + @Test + public void testSettingOperatorUidHash() { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + final DataStreamSource src = env.fromElements(1, 2); + final String writerHash = "f6b178ce445dc3ffaa06bad27a51fead"; + final String committerHash = "68ac8ae79eae4e3135a54f9689c4aa10"; + final String globalCommitterHash = "77e6aa6eeb1643b3765e1e4a7a672f37"; + final CustomSinkOperatorUidHashes operatorsUidHashes = + CustomSinkOperatorUidHashes.builder() + .setWriterUidHash(writerHash) + .setCommitterUidHash(committerHash) + .setGlobalCommitterUidHash(globalCommitterHash) + .build(); + src.sinkTo( + TestSink.newBuilder() + .setDefaultCommitter() + .setDefaultGlobalCommitter() + .build(), + operatorsUidHashes) + .name(NAME); + + final StreamGraph streamGraph = env.getStreamGraph(); + + assertEquals(findWriter(streamGraph).getUserHash(), writerHash); + assertEquals(findCommitter(streamGraph).getUserHash(), committerHash); + assertEquals(findGlobalCommitter(streamGraph).getUserHash(), globalCommitterHash); + } + + /** + * When ever you need to change something in this test case please think about possible state + * upgrade problems introduced by your changes. + */ + @Test + public void testSettingOperatorUids() { + final String sinkUid = "f6b178ce445dc3ffaa06bad27a51fead"; + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + final DataStreamSource src = env.fromElements(1, 2); + src.sinkTo(TestSink.newBuilder().setDefaultCommitter().setDefaultGlobalCommitter().build()) + .name(NAME) + .uid(sinkUid); + + final StreamGraph streamGraph = env.getStreamGraph(); + assertEquals(findWriter(streamGraph).getTransformationUID(), sinkUid); + assertEquals( + findCommitter(streamGraph).getTransformationUID(), + String.format("Sink Committer: %s", sinkUid)); + assertEquals( + findGlobalCommitter(streamGraph).getTransformationUID(), + String.format("Sink %s Global Committer", sinkUid)); + } + private void validateTopology( StreamNode src, Class srcOutTypeInfo, diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/source/SourceOperatorEventTimeTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/source/SourceOperatorEventTimeTest.java index 41b1bdfb3a8ee..959bc82402f6a 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/source/SourceOperatorEventTimeTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/source/SourceOperatorEventTimeTest.java @@ -18,30 +18,17 @@ package org.apache.flink.streaming.api.operators.source; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.state.OperatorStateStore; import org.apache.flink.api.connector.source.ReaderOutput; import org.apache.flink.api.connector.source.SourceReader; import org.apache.flink.api.connector.source.mocks.MockSourceSplit; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.api.connector.source.mocks.MockSourceSplitSerializer; import org.apache.flink.core.io.InputStatus; -import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; -import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; -import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.runtime.state.StateInitializationContextImpl; -import org.apache.flink.runtime.state.TestTaskStateManager; -import org.apache.flink.runtime.state.memory.MemoryStateBackend; +import org.apache.flink.runtime.source.event.AddSplitEvent; import org.apache.flink.streaming.api.operators.SourceOperator; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.io.DataInputStatus; -import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; -import org.apache.flink.streaming.runtime.tasks.SourceOperatorStreamTask; -import org.apache.flink.streaming.runtime.tasks.StreamMockEnvironment; import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; -import org.apache.flink.streaming.util.MockOutput; -import org.apache.flink.streaming.util.MockStreamConfig; import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; @@ -52,13 +39,13 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.function.Consumer; import java.util.stream.Collectors; +import static org.apache.flink.streaming.api.operators.source.TestingSourceOperator.createTestOperator; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.hasSize; import static org.junit.Assert.assertThat; @@ -159,6 +146,35 @@ public void testPerSplitOutputEventWatermarks() throws Exception { result, new Watermark(100L), new Watermark(150L), new Watermark(200L)); } + @Test + public void testCreatingPerSplitOutputOnSplitAddition() throws Exception { + final WatermarkStrategy watermarkStrategy = + WatermarkStrategy.forGenerator((ctx) -> new OnEventTestWatermarkGenerator<>()); + + InterpretingSourceReader reader = + new InterpretingSourceReader( + // No watermark (no record from split 2, whose watermark is Long.MIN_VALUE) + (output) -> output.createOutputForSplit("1").collect(0, 100L), + (output) -> output.createOutputForSplit("1").collect(0, 200L), + (output) -> output.createOutputForSplit("1").collect(0, 300L), + // Emit watermark 150 (from the 1st record of split 2) + (output) -> output.createOutputForSplit("2").collect(0, 150L), + // Emit watermark 300 (from the 3rd record in split 1) + (output) -> output.createOutputForSplit("2").collect(0, 400L)); + SourceOperator sourceOperator = + createTestOperator(reader, watermarkStrategy, emitProgressiveWatermarks); + + // Add two splits to SourceOperator. Output for two splits should be created during event + // handling. + sourceOperator.handleOperatorEvent( + new AddSplitEvent<>( + Arrays.asList(new MockSourceSplit(1), new MockSourceSplit(2)), + new MockSourceSplitSerializer())); + + final List result = testSequenceOfWatermarks(sourceOperator); + assertWatermarksOrEmpty(result, new Watermark(150L), new Watermark(300L)); + } + // ------------------------------------------------------------------------ // test execution helpers // ------------------------------------------------------------------------ @@ -186,9 +202,18 @@ private final List testSequenceOfWatermarks( final WatermarkStrategy watermarkStrategy, final Consumer>... actions) throws Exception { + final SourceReader reader = new InterpretingSourceReader(actions); + final SourceOperator sourceOperator = + createTestOperator(reader, watermarkStrategy, emitProgressiveWatermarks); - final List allEvents = - testSequenceOfEvents(emitProgressiveWatermarks, watermarkStrategy, actions); + return testSequenceOfWatermarks(sourceOperator); + } + + @SuppressWarnings("FinalPrivateMethod") + private final List testSequenceOfWatermarks( + SourceOperator sourceOperator) throws Exception { + + final List allEvents = testSequenceOfEvents(sourceOperator); return allEvents.stream() .filter((evt) -> evt instanceof Watermark) @@ -197,23 +222,13 @@ private final List testSequenceOfWatermarks( } @SuppressWarnings("FinalPrivateMethod") - @SafeVarargs private final List testSequenceOfEvents( - final boolean emitProgressiveWatermarks, - final WatermarkStrategy watermarkStrategy, - final Consumer>... actions) - throws Exception { + final SourceOperator sourceOperator) throws Exception { final CollectingDataOutput out = new CollectingDataOutput<>(); - final TestProcessingTimeService timeService = new TestProcessingTimeService(); - timeService.setCurrentTime(Integer.MAX_VALUE); // start somewhere that is not zero - - final SourceReader reader = new InterpretingSourceReader(actions); - - final SourceOperator sourceOperator = - createTestOperator( - reader, watermarkStrategy, timeService, emitProgressiveWatermarks); + final TestProcessingTimeService timeService = + ((TestProcessingTimeService) sourceOperator.getProcessingTimeService()); while (sourceOperator.emitNext(out) != DataInputStatus.END_OF_INPUT) { timeService.setCurrentTime(timeService.getCurrentProcessingTime() + 100); @@ -222,50 +237,6 @@ private final List testSequenceOfEvents( return out.events; } - // ------------------------------------------------------------------------ - // test setup helpers - // ------------------------------------------------------------------------ - - private static SourceOperator createTestOperator( - SourceReader reader, - WatermarkStrategy watermarkStrategy, - ProcessingTimeService timeService, - boolean emitProgressiveWatermarks) - throws Exception { - - final OperatorStateStore operatorStateStore = - new MemoryStateBackend() - .createOperatorStateBackend( - new MockEnvironmentBuilder().build(), - "test-operator", - Collections.emptyList(), - new CloseableRegistry()); - - final StateInitializationContext stateContext = - new StateInitializationContextImpl(null, operatorStateStore, null, null, null); - - final SourceOperator sourceOperator = - new TestingSourceOperator<>( - reader, watermarkStrategy, timeService, emitProgressiveWatermarks); - - sourceOperator.setup( - new SourceOperatorStreamTask( - new StreamMockEnvironment( - new Configuration(), - new Configuration(), - new ExecutionConfig(), - 1L, - new MockInputSplitProvider(), - 1, - new TestTaskStateManager())), - new MockStreamConfig(new Configuration(), 1), - new MockOutput<>(new ArrayList<>())); - sourceOperator.initializeState(stateContext); - sourceOperator.open(); - - return sourceOperator; - } - // ------------------------------------------------------------------------ // test mocks // ------------------------------------------------------------------------ diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/source/TestingSourceOperator.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/source/TestingSourceOperator.java index 5fbfd7c98b7fa..d25226e9bdd2f 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/source/TestingSourceOperator.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/source/TestingSourceOperator.java @@ -20,18 +20,34 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.state.OperatorStateStore; import org.apache.flink.api.connector.source.SourceReader; import org.apache.flink.api.connector.source.mocks.MockSourceSplit; import org.apache.flink.api.connector.source.mocks.MockSourceSplitSerializer; import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.operators.coordination.MockOperatorEventGateway; import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; +import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; +import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateInitializationContextImpl; +import org.apache.flink.runtime.state.TestTaskStateManager; +import org.apache.flink.runtime.state.hashmap.HashMapStateBackend; import org.apache.flink.streaming.api.operators.SourceOperator; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.streaming.runtime.tasks.SourceOperatorStreamTask; +import org.apache.flink.streaming.runtime.tasks.StreamMockEnvironment; +import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; +import org.apache.flink.streaming.util.MockOutput; +import org.apache.flink.streaming.util.MockStreamConfig; import org.apache.flink.streaming.util.MockStreamingRuntimeContext; +import java.util.ArrayList; +import java.util.Collections; + /** A SourceOperator extension to simplify test setup. */ public class TestingSourceOperator extends SourceOperator { @@ -100,4 +116,46 @@ public ExecutionConfig getExecutionConfig() { cfg.setAutoWatermarkInterval(100); return cfg; } + + public static SourceOperator createTestOperator( + SourceReader reader, + WatermarkStrategy watermarkStrategy, + boolean emitProgressiveWatermarks) + throws Exception { + + final OperatorStateStore operatorStateStore = + new HashMapStateBackend() + .createOperatorStateBackend( + new MockEnvironmentBuilder().build(), + "test-operator", + Collections.emptyList(), + new CloseableRegistry()); + + final StateInitializationContext stateContext = + new StateInitializationContextImpl(null, operatorStateStore, null, null, null); + + TestProcessingTimeService timeService = new TestProcessingTimeService(); + timeService.setCurrentTime(Integer.MAX_VALUE); // start somewhere that is not zero + + final SourceOperator sourceOperator = + new TestingSourceOperator<>( + reader, watermarkStrategy, timeService, emitProgressiveWatermarks); + + sourceOperator.setup( + new SourceOperatorStreamTask( + new StreamMockEnvironment( + new Configuration(), + new Configuration(), + new ExecutionConfig(), + 1L, + new MockInputSplitProvider(), + 1, + new TestTaskStateManager())), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(new ArrayList<>())); + sourceOperator.initializeState(stateContext); + sourceOperator.open(); + + return sourceOperator; + } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TimestampsAndWatermarksOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TimestampsAndWatermarksOperatorTest.java index 555a1a698c2ee..b7f34619a3024 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TimestampsAndWatermarksOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TimestampsAndWatermarksOperatorTest.java @@ -25,6 +25,7 @@ import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.junit.Test; @@ -56,6 +57,19 @@ public void inputWatermarksAreNotForwarded() throws Exception { assertThat(testHarness.getOutput(), empty()); } + @Test + public void inputStatusesAreNotForwarded() throws Exception { + OneInputStreamOperatorTestHarness testHarness = + createTestHarness( + WatermarkStrategy.forGenerator((ctx) -> new PeriodicWatermarkGenerator()) + .withTimestampAssigner((ctx) -> new LongExtractor())); + + testHarness.processWatermarkStatus(WatermarkStatus.IDLE); + testHarness.setProcessingTime(AUTO_WATERMARK_INTERVAL); + + assertThat(testHarness.getOutput(), empty()); + } + @Test public void longMaxInputWatermarkIsForwarded() throws Exception { OneInputStreamOperatorTestHarness testHarness = diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTest.java index 57a4367851496..0be5ed7a8c55b 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTest.java @@ -29,23 +29,38 @@ public class TimeWindowTest { @Test public void testGetWindowStartWithOffset() { - // [0, 7), [7, 14), [14, 21)... + // [-21, -14), [-14, -7), [-7, 0), [0, 7), [7, 14), [14, 21)... long offset = 0; + Assert.assertEquals(TimeWindow.getWindowStartWithOffset(-8, offset, 7), -14); + Assert.assertEquals(TimeWindow.getWindowStartWithOffset(-7, offset, 7), -7); + Assert.assertEquals(TimeWindow.getWindowStartWithOffset(-6, offset, 7), -7); + Assert.assertEquals(TimeWindow.getWindowStartWithOffset(-1, offset, 7), -7); Assert.assertEquals(TimeWindow.getWindowStartWithOffset(1, offset, 7), 0); Assert.assertEquals(TimeWindow.getWindowStartWithOffset(6, offset, 7), 0); Assert.assertEquals(TimeWindow.getWindowStartWithOffset(7, offset, 7), 7); Assert.assertEquals(TimeWindow.getWindowStartWithOffset(8, offset, 7), 7); - // [-4, 3), [3, 10), [10, 17)... + // [-11, -4), [-4, 3), [3, 10), [10, 17)... offset = 3; + Assert.assertEquals(TimeWindow.getWindowStartWithOffset(-10, offset, 7), -11); + Assert.assertEquals(TimeWindow.getWindowStartWithOffset(-9, offset, 7), -11); + Assert.assertEquals(TimeWindow.getWindowStartWithOffset(-3, offset, 7), -4); + Assert.assertEquals(TimeWindow.getWindowStartWithOffset(-2, offset, 7), -4); + Assert.assertEquals(TimeWindow.getWindowStartWithOffset(-1, offset, 7), -4); Assert.assertEquals(TimeWindow.getWindowStartWithOffset(1, offset, 7), -4); Assert.assertEquals(TimeWindow.getWindowStartWithOffset(2, offset, 7), -4); Assert.assertEquals(TimeWindow.getWindowStartWithOffset(3, offset, 7), 3); Assert.assertEquals(TimeWindow.getWindowStartWithOffset(9, offset, 7), 3); Assert.assertEquals(TimeWindow.getWindowStartWithOffset(10, offset, 7), 10); - // [-2, 5), [5, 12), [12, 19)... + // [-16, -9), [-9, -2), [-2, 5), [5, 12), [12, 19)... offset = -2; + Assert.assertEquals(TimeWindow.getWindowStartWithOffset(-12, offset, 7), -16); + Assert.assertEquals(TimeWindow.getWindowStartWithOffset(-7, offset, 7), -9); + Assert.assertEquals(TimeWindow.getWindowStartWithOffset(-4, offset, 7), -9); + Assert.assertEquals(TimeWindow.getWindowStartWithOffset(-3, offset, 7), -9); + Assert.assertEquals(TimeWindow.getWindowStartWithOffset(2, offset, 7), -2); + Assert.assertEquals(TimeWindow.getWindowStartWithOffset(-1, offset, 7), -2); Assert.assertEquals(TimeWindow.getWindowStartWithOffset(1, offset, 7), -2); Assert.assertEquals(TimeWindow.getWindowStartWithOffset(-2, offset, 7), -2); Assert.assertEquals(TimeWindow.getWindowStartWithOffset(3, offset, 7), -2); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTaskTest.java index 3d9087f55d2b4..d3b9f64de26d9 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTaskTest.java @@ -38,6 +38,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.checkpoint.CheckpointType; import org.apache.flink.runtime.checkpoint.SavepointType; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -59,7 +60,10 @@ import org.apache.flink.streaming.runtime.tasks.LifeCycleMonitor.LifeCyclePhase; import org.apache.flink.util.SerializedValue; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import java.io.IOException; import java.io.Serializable; @@ -73,31 +77,31 @@ import java.util.concurrent.Future; import java.util.function.Supplier; import java.util.stream.IntStream; +import java.util.stream.Stream; import static org.apache.flink.streaming.util.TestHarnessUtil.assertOutputEquals; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.contains; -import static org.hamcrest.Matchers.hasSize; -import static org.hamcrest.Matchers.lessThanOrEqualTo; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** * Tests for verifying that the {@link SourceOperator} as a task input can be integrated well with * {@link org.apache.flink.streaming.runtime.io.StreamOneInputProcessor}. */ -public class SourceOperatorStreamTaskTest extends SourceStreamTaskTestBase { +class SourceOperatorStreamTaskTest extends SourceStreamTaskTestBase { + private static final OperatorID OPERATOR_ID = new OperatorID(); private static final int NUM_RECORDS = 10; + public static final CheckpointStorageLocationReference SAVEPOINT_LOCATION = + new CheckpointStorageLocationReference("Savepoint".getBytes()); + public static final CheckpointStorageLocationReference CHECKPOINT_LOCATION = + new CheckpointStorageLocationReference("Checkpoint".getBytes()); @Test - public void testMetrics() throws Exception { + void testMetrics() throws Exception { testMetrics( SourceOperatorStreamTask::new, new SourceOperatorFactory<>( new MockSource(Boundedness.BOUNDED, 1), WatermarkStrategy.noWatermarks()), - lessThanOrEqualTo(1_000_000d)); + busyTime -> busyTime.isLessThanOrEqualTo(1_000_000d)); } /** @@ -105,7 +109,7 @@ public void testMetrics() throws Exception { * operators. */ @Test - public void testSnapshotAndRestore() throws Exception { + void testSnapshotAndRestore() throws Exception { // process NUM_RECORDS records and take a snapshot. TaskStateSnapshot taskStateSnapshot = executeAndWaitForCheckpoint(1, null, IntStream.range(0, NUM_RECORDS)); @@ -116,7 +120,7 @@ public void testSnapshotAndRestore() throws Exception { } @Test - public void testSnapshotAndAdvanceToEndOfEventTime() throws Exception { + void testSnapshotAndAdvanceToEndOfEventTime() throws Exception { final int checkpointId = 1; try (StreamTaskMailboxTestHarness testHarness = createTestHarness(checkpointId, null)) { @@ -139,7 +143,7 @@ public void testSnapshotAndAdvanceToEndOfEventTime() throws Exception { } @Test - public void testEmittingMaxWatermarkAfterReadingAllRecords() throws Exception { + void testEmittingMaxWatermarkAfterReadingAllRecords() throws Exception { try (StreamTaskMailboxTestHarness testHarness = createTestHarness()) { testHarness.processAll(); testHarness.finishProcessing(); @@ -147,22 +151,49 @@ public void testEmittingMaxWatermarkAfterReadingAllRecords() throws Exception { Queue expectedOutput = new LinkedList<>(); expectedOutput.add(Watermark.MAX_WATERMARK); expectedOutput.add(new EndOfData(StopMode.DRAIN)); - assertThat(testHarness.getOutput().toArray(), equalTo(expectedOutput.toArray())); + assertThat(testHarness.getOutput().toArray()).isEqualTo(expectedOutput.toArray()); } } @Test - public void testNotEmittingMaxWatermarkAfterCancelling() throws Exception { + void testNotEmittingMaxWatermarkAfterCancelling() throws Exception { try (StreamTaskMailboxTestHarness testHarness = createTestHarness()) { testHarness.getStreamTask().cancel(); testHarness.finishProcessing(); - assertThat(testHarness.getOutput(), hasSize(0)); + assertThat(testHarness.getOutput()).hasSize(0); } } - @Test - public void testExternallyInducedSource() throws Exception { + static Stream provideExternallyInducedParameters() { + return Stream.of( + CheckpointOptions.alignedNoTimeout( + SavepointType.savepoint(SavepointFormatType.CANONICAL), + SAVEPOINT_LOCATION), + CheckpointOptions.alignedNoTimeout( + SavepointType.terminate(SavepointFormatType.CANONICAL), + SAVEPOINT_LOCATION), + CheckpointOptions.alignedNoTimeout( + SavepointType.suspend(SavepointFormatType.CANONICAL), + SAVEPOINT_LOCATION), + CheckpointOptions.alignedNoTimeout( + CheckpointType.CHECKPOINT, CHECKPOINT_LOCATION), + CheckpointOptions.alignedWithTimeout( + CheckpointType.CHECKPOINT, CHECKPOINT_LOCATION, 123L), + CheckpointOptions.unaligned(CheckpointType.CHECKPOINT, CHECKPOINT_LOCATION), + CheckpointOptions.notExactlyOnce( + CheckpointType.CHECKPOINT, CHECKPOINT_LOCATION)) + .flatMap( + options -> + Stream.of( + new Object[] {options, true}, + new Object[] {options, false})); + } + + @ParameterizedTest + @MethodSource("provideExternallyInducedParameters") + void testExternallyInducedSource(CheckpointOptions checkpointOptions, boolean rpcFirst) + throws Exception { final int numEventsBeforeCheckpoint = 10; final int totalNumEvents = 20; TestingExternallyInducedSourceReader testingReader = @@ -174,19 +205,52 @@ public void testExternallyInducedSource() throws Exception { ((SourceOperator) testHarness.getStreamTask().mainOperator) .getSourceReader(); - testHarness.processAll(); + CheckpointMetaData checkpointMetaData = + new CheckpointMetaData(TestingExternallyInducedSourceReader.CHECKPOINT_ID, 2); + if (rpcFirst) { + testHarness.streamTask.triggerCheckpointAsync( + checkpointMetaData, checkpointOptions); + testHarness.processAll(); + } else { + do { + testHarness.processSingleStep(); + } while (!runtimeTestingReader.shouldTriggerCheckpoint().isPresent()); + // stream task should block when trigger received but no RPC + assertThat(testHarness.streamTask.inputProcessor.isAvailable()).isFalse(); + CompletableFuture triggerCheckpointAsync = + testHarness.streamTask.triggerCheckpointAsync( + checkpointMetaData, checkpointOptions); + // process mails until checkpoint has been processed + while (!triggerCheckpointAsync.isDone()) { + testHarness.processSingleStep(); + } + // stream task should be unblocked now + assertThat(testHarness.streamTask.inputProcessor.isAvailable()).isTrue(); + testHarness.processAll(); + } - assertEquals(totalNumEvents, runtimeTestingReader.numEmittedEvents); - assertTrue(runtimeTestingReader.checkpointed); - assertEquals( - TestingExternallyInducedSourceReader.CHECKPOINT_ID, - runtimeTestingReader.checkpointedId); - assertEquals(numEventsBeforeCheckpoint, runtimeTestingReader.checkpointedAt); + int expectedEvents = + checkpointOptions.getCheckpointType().isSavepoint() + && ((SavepointType) checkpointOptions.getCheckpointType()) + .isSynchronous() + ? numEventsBeforeCheckpoint + : totalNumEvents; + assertThat(runtimeTestingReader.numEmittedEvents).isEqualTo(expectedEvents); + assertThat(runtimeTestingReader.checkpointed).isTrue(); + assertThat(runtimeTestingReader.checkpointedId) + .isEqualTo(TestingExternallyInducedSourceReader.CHECKPOINT_ID); + assertThat(runtimeTestingReader.checkpointedAt).isEqualTo(numEventsBeforeCheckpoint); + Assertions.assertThat(testHarness.getOutput()) + .contains( + new CheckpointBarrier( + checkpointMetaData.getCheckpointId(), + checkpointMetaData.getTimestamp(), + checkpointOptions)); } } @Test - public void testSkipExecutionIfFinishedOnRestore() throws Exception { + void testSkipExecutionIfFinishedOnRestore() throws Exception { TaskStateSnapshot taskStateSnapshot = TaskStateSnapshot.FINISHED_ON_RESTORE; LifeCycleMonitorSource testingSource = @@ -215,7 +279,8 @@ public void notifyEndOfData(StopMode mode) throws IOException { testHarness.getStreamTask().invoke(); testHarness.processAll(); - assertThat(output, contains(Watermark.MAX_WATERMARK, new EndOfData(StopMode.DRAIN))); + assertThat(output) + .containsExactly(Watermark.MAX_WATERMARK, new EndOfData(StopMode.DRAIN)); LifeCycleMonitorSourceReader sourceReader = (LifeCycleMonitorSourceReader) @@ -226,7 +291,7 @@ public void notifyEndOfData(StopMode mode) throws IOException { } @Test - public void testTriggeringStopWithSavepointWithDrain() throws Exception { + void testTriggeringStopWithSavepointWithDrain() throws Exception { SourceOperatorFactory sourceOperatorFactory = new SourceOperatorFactory<>( new MockSource(Boundedness.CONTINUOUS_UNBOUNDED, 2), @@ -264,16 +329,16 @@ public void acknowledgeCheckpoint( new CheckpointMetaData(2, 2), CheckpointOptions.alignedNoTimeout( SavepointType.terminate(SavepointFormatType.CANONICAL), - CheckpointStorageLocationReference.getDefault())); + SAVEPOINT_LOCATION)); checkpointCompleted.whenComplete( (ignored, exception) -> testHarness.streamTask.notifyCheckpointCompleteAsync(2)); testHarness.waitForTaskCompletion(); testHarness.finishProcessing(); - assertTrue(triggerResult.isDone()); - assertTrue(triggerResult.get()); - assertTrue(checkpointCompleted.isDone()); + assertThat(triggerResult.isDone()).isTrue(); + assertThat(triggerResult.get()).isTrue(); + assertThat(checkpointCompleted.isDone()).isTrue(); } } @@ -304,7 +369,8 @@ private TaskStateSnapshot executeAndWaitForCheckpoint( expectedOutput.add( new CheckpointBarrier(checkpointId, checkpointId, checkpointOptions)); - assertEquals(checkpointId, testHarness.taskStateManager.getReportedCheckpointId()); + assertThat(testHarness.taskStateManager.getReportedCheckpointId()) + .isEqualTo(checkpointId); assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); return testHarness.taskStateManager.getLastJobManagerTaskStateSnapshot(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java index aa4b364cbc113..d16c5abc2a476 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java @@ -70,8 +70,7 @@ import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.function.CheckedSupplier; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.io.Serializable; @@ -94,23 +93,18 @@ import static org.apache.flink.api.common.typeinfo.BasicTypeInfo.STRING_TYPE_INFO; import static org.apache.flink.streaming.runtime.tasks.StreamTaskFinalCheckpointsTest.triggerCheckpoint; import static org.apache.flink.util.Preconditions.checkState; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.contains; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; /** * These tests verify that the RichFunction methods are called (in correct order). And that * checkpointing/element emission don't occur concurrently. */ -public class SourceStreamTaskTest extends SourceStreamTaskTestBase { +class SourceStreamTaskTest extends SourceStreamTaskTestBase { /** This test verifies that open() and close() are correctly called by the StreamTask. */ @Test - public void testOpenClose() throws Exception { + void testOpenClose() throws Exception { final StreamTaskTestHarness testHarness = new StreamTaskTestHarness<>(SourceStreamTask::new, STRING_TYPE_INFO); @@ -124,15 +118,17 @@ public void testOpenClose() throws Exception { testHarness.invoke(); testHarness.waitForTaskCompletion(); - assertTrue("RichFunction methods where not called.", OpenCloseTestSource.closeCalled); + assertThat(OpenCloseTestSource.closeCalled) + .as("RichFunction methods where not called.") + .isTrue(); List resultElements = TestHarnessUtil.getRawElementsFromOutput(testHarness.getOutput()); - Assert.assertEquals(10, resultElements.size()); + assertThat(resultElements.size()).isEqualTo(10); } @Test - public void testMetrics() throws Exception { + void testMetrics() throws Exception { testMetrics( SourceStreamTask::new, SimpleOperatorFactory.of( @@ -140,7 +136,7 @@ public void testMetrics() throws Exception { new CancelTestSource( INT_TYPE_INFO.createSerializer(new ExecutionConfig()), 42))), - is(Double.NaN)); + busyTime -> busyTime.isNaN()); } /** @@ -157,7 +153,7 @@ public void testMetrics() throws Exception { */ @Test @SuppressWarnings("unchecked") - public void testCheckpointing() throws Exception { + void testCheckpointing() throws Exception { final int numElements = 100; final int numCheckpoints = 100; final int numCheckpointers = 1; @@ -214,14 +210,14 @@ public void testCheckpointing() throws Exception { List> resultElements = TestHarnessUtil.getRawElementsFromOutput(testHarness.getOutput()); - Assert.assertEquals(numElements, resultElements.size()); + assertThat(resultElements.size()).isEqualTo(numElements); } finally { executor.shutdown(); } } @Test - public void testClosingAllOperatorsOnChainProperly() throws Exception { + void testClosingAllOperatorsOnChainProperly() throws Exception { final StreamTaskTestHarness testHarness = new StreamTaskTestHarness<>(SourceStreamTask::new, STRING_TYPE_INFO); @@ -254,11 +250,11 @@ public void testClosingAllOperatorsOnChainProperly() throws Exception { new StreamRecord<>("[Operator1]: Finish")); final Object[] output = testHarness.getOutput().toArray(); - assertArrayEquals("Output was not correct.", expected.toArray(), output); + assertThat(output).as("Output was not correct.").isEqualTo(expected.toArray()); } @Test - public void testNotMarkingEndOfInputWhenTaskCancelled() throws Exception { + void testNotMarkingEndOfInputWhenTaskCancelled() throws Exception { final StreamTaskTestHarness testHarness = new StreamTaskTestHarness<>(SourceStreamTask::new, STRING_TYPE_INFO); @@ -299,22 +295,22 @@ public void testNotMarkingEndOfInputWhenTaskCancelled() throws Exception { } @Test - public void testCancellationWithSourceBlockedOnLock() throws Exception { + void testCancellationWithSourceBlockedOnLock() throws Exception { testCancellationWithSourceBlockedOnLock(false, false); } @Test - public void testCancellationWithSourceBlockedOnLockWithPendingMail() throws Exception { + void testCancellationWithSourceBlockedOnLockWithPendingMail() throws Exception { testCancellationWithSourceBlockedOnLock(true, false); } @Test - public void testCancellationWithSourceBlockedOnLockAndThrowingOnError() throws Exception { + void testCancellationWithSourceBlockedOnLockAndThrowingOnError() throws Exception { testCancellationWithSourceBlockedOnLock(false, true); } @Test - public void testCancellationWithSourceBlockedOnLockWithPendingMailAndThrowingOnError() + void testCancellationWithSourceBlockedOnLockWithPendingMailAndThrowingOnError() throws Exception { testCancellationWithSourceBlockedOnLock(true, true); } @@ -324,8 +320,8 @@ public void testCancellationWithSourceBlockedOnLockWithPendingMailAndThrowingOnE * StreamTask} which, as of the time this test is being written, is not tested anywhere else * (like {@link StreamTaskTest} or {@link OneInputStreamTaskTest}). */ - public void testCancellationWithSourceBlockedOnLock( - boolean withPendingMail, boolean throwInCancel) throws Exception { + void testCancellationWithSourceBlockedOnLock(boolean withPendingMail, boolean throwInCancel) + throws Exception { final StreamTaskTestHarness testHarness = new StreamTaskTestHarness<>(SourceStreamTask::new, STRING_TYPE_INFO); @@ -354,9 +350,9 @@ public void testCancellationWithSourceBlockedOnLock( .createExecutor(0) .execute( () -> - assertFalse( - "This should never execute before task cancelation", - testHarness.getTask().isRunning()), + assertThat(testHarness.getTask().isRunning()) + .as("This should never execute before task cancelation") + .isFalse(), "Test"); } @@ -427,12 +423,12 @@ public void cancel() { } @Test - public void testInterruptionExceptionNotSwallowed() throws Exception { + void testInterruptionExceptionNotSwallowed() throws Exception { testInterruptionExceptionNotSwallowed(InterruptedException::new); } @Test - public void testWrappedInterruptionExceptionNotSwallowed() throws Exception { + void testWrappedInterruptionExceptionNotSwallowed() throws Exception { testInterruptionExceptionNotSwallowed( () -> new RuntimeException(new FlinkRuntimeException(new InterruptedException()))); } @@ -491,7 +487,7 @@ public void cancel() {} } @Test - public void testWaitsForSourceThreadOnCancel() throws Exception { + void testWaitsForSourceThreadOnCancel() throws Exception { StreamTaskTestHarness harness = new StreamTaskTestHarness<>(SourceStreamTask::new, STRING_TYPE_INFO); @@ -504,13 +500,13 @@ public void testWaitsForSourceThreadOnCancel() throws Exception { // SourceStreamTask should be still waiting for NonStoppingSource after cancellation harness.getTask().cancel(); harness.waitForTaskCompletion(50, true); // allow task to exit prematurely - assertTrue(harness.taskThread.isAlive()); + assertThat(harness.taskThread.isAlive()).isTrue(); // SourceStreamTask should be still waiting for NonStoppingSource after interruptions for (int i = 0; i < 10; i++) { harness.getTask().maybeInterruptOnCancel(harness.getTaskThread(), null, null); harness.waitForTaskCompletion(50, true); // allow task to exit prematurely - assertTrue(harness.taskThread.isAlive()); + assertThat(harness.taskThread.isAlive()).isTrue(); } // It should only exit once NonStoppingSource allows for it @@ -519,7 +515,7 @@ public void testWaitsForSourceThreadOnCancel() throws Exception { } @Test - public void testTriggeringCheckpointAfterSourceThreadFinished() throws Exception { + void testTriggeringCheckpointAfterSourceThreadFinished() throws Exception { ResultPartition[] partitionWriters = new ResultPartition[2]; try (NettyShuffleEnvironment env = new NettyShuffleEnvironmentBuilder() @@ -585,12 +581,12 @@ public void acknowledgeCheckpoint( (id, error) -> testHarness.getStreamTask().notifyCheckpointCompleteAsync(2)); testHarness.finishProcessing(); - assertTrue(checkpointFuture.isDone()); + assertThat(checkpointFuture.isDone()).isTrue(); // Each result partition should have emitted 1 barrier, 1 max watermark and 1 // EndOfUserRecordEvent. for (ResultPartition resultPartition : partitionWriters) { - assertEquals(3, resultPartition.getNumberOfQueuedBuffers()); + assertThat(resultPartition.getNumberOfQueuedBuffers()).isEqualTo(3); } } } finally { @@ -603,7 +599,7 @@ public void acknowledgeCheckpoint( } @Test - public void testClosedOnRestoreSourceSkipExecution() throws Exception { + void testClosedOnRestoreSourceSkipExecution() throws Exception { LifeCycleMonitorSource testSource = new LifeCycleMonitorSource(); List output = new ArrayList<>(); try (StreamTaskMailboxTestHarness harness = @@ -626,7 +622,8 @@ public void notifyEndOfData(StopMode mode) throws IOException { harness.processAll(); harness.streamTask.getCompletionFuture().get(); - assertThat(output, contains(Watermark.MAX_WATERMARK, new EndOfData(StopMode.DRAIN))); + assertThat(output) + .containsExactly(Watermark.MAX_WATERMARK, new EndOfData(StopMode.DRAIN)); LifeCycleMonitorSource source = (LifeCycleMonitorSource) @@ -642,7 +639,7 @@ public void notifyEndOfData(StopMode mode) throws IOException { } @Test - public void testTriggeringStopWithSavepointWithDrain() throws Exception { + void testTriggeringStopWithSavepointWithDrain() throws Exception { SourceFunction testSource = new EmptySource(); CompletableFuture checkpointCompleted = new CompletableFuture<>(); @@ -684,9 +681,9 @@ public void acknowledgeCheckpoint( harness.streamTask.runMailboxLoop(); harness.finishProcessing(); - assertTrue(triggerResult.isDone()); - assertTrue(triggerResult.get()); - assertTrue(checkpointCompleted.isDone()); + assertThat(triggerResult.isDone()).isTrue(); + assertThat(triggerResult.get()).isTrue(); + assertThat(checkpointCompleted.isDone()).isTrue(); } } @@ -767,7 +764,7 @@ public void cancel() { public List snapshotState(long checkpointId, long timestamp) throws Exception { if (!semaphore.tryAcquire()) { - Assert.fail("Concurrent invocation of snapshotState."); + fail("Concurrent invocation of snapshotState."); } int startCount = count; lastCheckpointId = checkpointId; @@ -780,7 +777,7 @@ public List snapshotState(long checkpointId, long timestamp) if (startCount != count) { semaphore.release(); // This means that next() was invoked while the snapshot was ongoing - Assert.fail("Count is different at start end end of snapshot."); + fail("Count is different at start end end of snapshot."); } semaphore.release(); return Collections.singletonList(sum); @@ -871,7 +868,7 @@ private static class OpenCloseTestSource extends RichSourceFunction { public void open(Configuration parameters) throws Exception { super.open(parameters); if (closeCalled) { - Assert.fail("Close called before open."); + fail("Close called before open."); } openCalled = true; } @@ -880,7 +877,7 @@ public void open(Configuration parameters) throws Exception { public void close() throws Exception { super.close(); if (!openCalled) { - Assert.fail("Open was not called before close."); + fail("Open was not called before close."); } closeCalled = true; } @@ -888,7 +885,7 @@ public void close() throws Exception { @Override public void run(SourceContext ctx) throws Exception { if (!openCalled) { - Assert.fail("Open was not called before run."); + fail("Open was not called before run."); } for (int i = 0; i < 10; i++) { ctx.collect("Hello" + i); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTestBase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTestBase.java index 30656413d551f..b477796704a76 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTestBase.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTestBase.java @@ -32,16 +32,15 @@ import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables; -import org.hamcrest.Matcher; +import org.assertj.core.api.AbstractDoubleAssert; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Future; +import java.util.function.Consumer; import static org.apache.flink.api.common.typeinfo.BasicTypeInfo.INT_TYPE_INFO; -import static org.hamcrest.Matchers.greaterThanOrEqualTo; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertThat; +import static org.assertj.core.api.Assertions.assertThat; /** Common base class for testing source tasks. */ public class SourceStreamTaskTestBase { @@ -49,7 +48,7 @@ public void testMetrics( FunctionWithException, Exception> taskFactory, StreamOperatorFactory operatorFactory, - Matcher busyTimeMatcher) + Consumer> busyTimeMatcher) throws Exception { long sleepTime = 42; @@ -73,26 +72,24 @@ public void testMetrics( OneShotLatch checkpointAcknowledgeLatch = new OneShotLatch(); harness.getCheckpointResponder().setAcknowledgeLatch(checkpointAcknowledgeLatch); - assertFalse(triggerFuture.isDone()); + assertThat(triggerFuture).isNotDone(); Thread.sleep(sleepTime); while (!triggerFuture.isDone()) { harness.streamTask.runMailboxStep(); } Gauge checkpointStartDelayGauge = (Gauge) metrics.get(MetricNames.CHECKPOINT_START_DELAY_TIME); - assertThat( - checkpointStartDelayGauge.getValue(), - greaterThanOrEqualTo(sleepTime * 1_000_000)); + assertThat(checkpointStartDelayGauge.getValue()) + .isGreaterThanOrEqualTo(sleepTime * 1_000_000); Gauge busyTimeGauge = (Gauge) metrics.get(MetricNames.TASK_BUSY_TIME); - assertThat(busyTimeGauge.getValue(), busyTimeMatcher); + busyTimeMatcher.accept(assertThat(busyTimeGauge.getValue())); checkpointAcknowledgeLatch.await(); TestCheckpointResponder.AcknowledgeReport acknowledgeReport = Iterables.getOnlyElement( harness.getCheckpointResponder().getAcknowledgeReports()); - assertThat( - acknowledgeReport.getCheckpointMetrics().getCheckpointStartDelayNanos(), - greaterThanOrEqualTo(sleepTime * 1_000_000)); + assertThat(acknowledgeReport.getCheckpointMetrics().getCheckpointStartDelayNanos()) + .isGreaterThanOrEqualTo(sleepTime * 1_000_000); } } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskFinalCheckpointsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskFinalCheckpointsTest.java index c85218d673d0e..08ab5d99e6025 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskFinalCheckpointsTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskFinalCheckpointsTest.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.core.execution.SavepointFormatType; @@ -62,7 +61,6 @@ import javax.annotation.Nullable; -import java.time.Duration; import java.util.Collections; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; @@ -886,8 +884,7 @@ public void testWaitingForUnalignedChannelStatesIfFinishedOnRestore() throws Exc CommonTestUtils.waitUntilCondition( () -> checkpointResponder.getAcknowledgeLatch().isTriggered() - || checkpointResponder.getDeclinedLatch().isTriggered(), - Deadline.fromNow(Duration.ofSeconds(10))); + || checkpointResponder.getDeclinedLatch().isTriggered()); assertEquals( Collections.singletonList(2L), diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java index a078a08751284..a65667d602e9a 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java @@ -30,6 +30,7 @@ import org.apache.flink.streaming.api.operators.StreamOperatorFactory; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; import org.apache.flink.util.Preconditions; import java.util.ArrayList; @@ -225,6 +226,16 @@ public void processWatermark(long watermark) throws Exception { processWatermark(new Watermark(watermark)); } + public void processWatermarkStatus(WatermarkStatus status) throws Exception { + if (inputs.isEmpty()) { + getOneInputOperator().processWatermarkStatus(status); + } else { + checkState(inputs.size() == 1); + Input input = inputs.get(0); + input.processWatermarkStatus(status); + } + } + public void processWatermark(Watermark mark) throws Exception { currentWatermark = mark.getTimestamp(); if (inputs.isEmpty()) { diff --git a/flink-streaming-scala/pom.xml b/flink-streaming-scala/pom.xml index bbee28d315c53..5b959c1e1d7b7 100644 --- a/flink-streaming-scala/pom.xml +++ b/flink-streaming-scala/pom.xml @@ -24,7 +24,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-sql-client/pom.xml b/flink-table/flink-sql-client/pom.xml index fbd61f6bb1665..b998f69d51c4a 100644 --- a/flink-table/flink-sql-client/pom.xml +++ b/flink-table/flink-sql-client/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/ExecutionContext.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/ExecutionContext.java index 135cf8f26f29b..4c0f7f4ee6a9e 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/ExecutionContext.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/ExecutionContext.java @@ -98,19 +98,20 @@ public StreamTableEnvironment getTableEnvironment() { // ------------------------------------------------------------------------------------------------------------------ private StreamTableEnvironment createTableEnvironment() { - // checks the value of RUNTIME_MODE - EnvironmentSettings settings = EnvironmentSettings.fromConfiguration(flinkConfig); + EnvironmentSettings settings = + EnvironmentSettings.newInstance().withConfiguration(flinkConfig).build(); - TableConfig tableConfig = new TableConfig(); - tableConfig.addConfiguration(flinkConfig); - - StreamExecutionEnvironment streamExecEnv = createStreamExecutionEnvironment(); + // We need not different StreamExecutionEnvironments to build and submit flink job, + // instead we just use StreamExecutionEnvironment#executeAsync(StreamGraph) method + // to execute existing StreamGraph. + // This requires StreamExecutionEnvironment to have a full flink configuration. + StreamExecutionEnvironment streamExecEnv = + new StreamExecutionEnvironment(new Configuration(flinkConfig), classLoader); - final Executor executor = lookupExecutor(settings.getExecutor(), streamExecEnv); + final Executor executor = lookupExecutor(streamExecEnv); return createStreamTableEnvironment( streamExecEnv, settings, - tableConfig, executor, sessionState.catalogManager, sessionState.moduleManager, @@ -121,21 +122,19 @@ private StreamTableEnvironment createTableEnvironment() { private StreamTableEnvironment createStreamTableEnvironment( StreamExecutionEnvironment env, EnvironmentSettings settings, - TableConfig tableConfig, Executor executor, CatalogManager catalogManager, ModuleManager moduleManager, FunctionCatalog functionCatalog, ClassLoader userClassLoader) { + TableConfig tableConfig = TableConfig.getDefault(); + tableConfig.setRootConfiguration(executor.getConfiguration()); + tableConfig.addConfiguration(settings.getConfiguration()); + final Planner planner = PlannerFactoryUtil.createPlanner( - settings.getPlanner(), - executor, - tableConfig, - moduleManager, - catalogManager, - functionCatalog); + executor, tableConfig, moduleManager, catalogManager, functionCatalog); return new StreamTableEnvironmentImpl( catalogManager, @@ -149,12 +148,11 @@ private StreamTableEnvironment createStreamTableEnvironment( userClassLoader); } - private Executor lookupExecutor( - String executorIdentifier, StreamExecutionEnvironment executionEnvironment) { + private Executor lookupExecutor(StreamExecutionEnvironment executionEnvironment) { try { final ExecutorFactory executorFactory = FactoryUtil.discoverFactory( - classLoader, ExecutorFactory.class, executorIdentifier); + classLoader, ExecutorFactory.class, ExecutorFactory.DEFAULT_IDENTIFIER); final Method createMethod = executorFactory .getClass() @@ -167,12 +165,4 @@ private Executor lookupExecutor( e); } } - - private StreamExecutionEnvironment createStreamExecutionEnvironment() { - // We need not different StreamExecutionEnvironments to build and submit flink job, - // instead we just use StreamExecutionEnvironment#executeAsync(StreamGraph) method - // to execute existing StreamGraph. - // This requires StreamExecutionEnvironment to have a full flink configuration. - return new StreamExecutionEnvironment(new Configuration(flinkConfig), classLoader); - } } diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/SessionContext.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/SessionContext.java index c45d5af79b702..c31f6f5f5191b 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/SessionContext.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/SessionContext.java @@ -221,7 +221,8 @@ public static SessionContext create(DefaultContext defaultContext, String sessio ModuleManager moduleManager = new ModuleManager(); - final EnvironmentSettings settings = EnvironmentSettings.fromConfiguration(configuration); + final EnvironmentSettings settings = + EnvironmentSettings.newInstance().withConfiguration(configuration).build(); CatalogManager catalogManager = CatalogManager.newBuilder() @@ -262,6 +263,9 @@ public void addJar(String jarPath) { } Set newDependencies = new HashSet<>(dependencies); + // merge the jars in config with the jars maintained in session + Set jarsInConfig = getJarsInConfig(); + newDependencies.addAll(jarsInConfig); newDependencies.add(jarURL); updateClassLoaderAndDependencies(newDependencies); @@ -280,6 +284,9 @@ public void removeJar(String jarPath) { } Set newDependencies = new HashSet<>(dependencies); + // merge the jars in config with the jars maintained in session + Set jarsInConfig = getJarsInConfig(); + newDependencies.addAll(jarsInConfig); newDependencies.remove(jarURL); updateClassLoaderAndDependencies(newDependencies); @@ -324,22 +331,10 @@ private void resetSessionConfigurationToDefault(Configuration defaultConf) { } private void updateClassLoaderAndDependencies(Collection newDependencies) { - // merge the jar in config with the jar maintained in session - Set jarsInConfig; - try { - jarsInConfig = - new HashSet<>( - ConfigUtils.decodeListFromConfig( - sessionConfiguration, PipelineOptions.JARS, URL::new)); - } catch (MalformedURLException e) { - throw new SqlExecutionException( - "Failed to parse the option `pipeline.jars` in configuration.", e); - } - jarsInConfig.addAll(newDependencies); ConfigUtils.encodeCollectionToConfig( sessionConfiguration, PipelineOptions.JARS, - new ArrayList<>(jarsInConfig), + new ArrayList<>(newDependencies), URL::toString); // TODO: update the classloader in CatalogManager. @@ -374,4 +369,18 @@ private URL getURLFromPath(String jarPath, String message) { e); } } + + private Set getJarsInConfig() { + Set jarsInConfig; + try { + jarsInConfig = + new HashSet<>( + ConfigUtils.decodeListFromConfig( + sessionConfiguration, PipelineOptions.JARS, URL::new)); + } catch (MalformedURLException e) { + throw new SqlExecutionException( + "Failed to parse the option `pipeline.jars` in configuration.", e); + } + return jarsInConfig; + } } diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientTest.java index 1aae9f3d077fa..f6fe6b6a567ea 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientTest.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientTest.java @@ -18,7 +18,6 @@ package org.apache.flink.table.client.cli; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.client.cli.DefaultCLI; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; @@ -64,7 +63,6 @@ import java.io.OutputStream; import java.nio.file.Files; import java.nio.file.Path; -import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -327,8 +325,7 @@ public void testCancelExecutionInteractiveMode() throws Exception { client.getTerminal().raise(Terminal.Signal.INT); CommonTestUtils.waitUntilCondition( - () -> outputStream.toString().contains("'key' = 'value'"), - Deadline.fromNow(Duration.ofMillis(10000))); + () -> outputStream.toString().contains("'key' = 'value'")); } } diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliResultViewTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliResultViewTest.java index a72fdf717340f..ddc1f5fba23e5 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliResultViewTest.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliResultViewTest.java @@ -21,7 +21,6 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.internal.TableResultInternal; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; @@ -139,8 +138,7 @@ private static final class MockExecutor implements Executor { private final TypedResult typedResult; private final CountDownLatch cancellationCounter; - private static final Configuration defaultConfig = - TableConfig.getDefault().getConfiguration(); + private static final Configuration defaultConfig = new Configuration(); public MockExecutor(TypedResult typedResult, CountDownLatch cancellationCounter) { this.typedResult = typedResult; diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliTableauResultViewTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliTableauResultViewTest.java index c0a5b1f095830..4deb8da86f86c 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliTableauResultViewTest.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliTableauResultViewTest.java @@ -19,7 +19,6 @@ package org.apache.flink.table.client.cli; import org.apache.flink.api.common.RuntimeExecutionMode; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.table.api.DataTypes; @@ -45,7 +44,6 @@ import java.io.ByteArrayOutputStream; import java.math.BigDecimal; import java.sql.Timestamp; -import java.time.Duration; import java.util.Arrays; import java.util.List; import java.util.concurrent.ExecutorService; @@ -82,7 +80,8 @@ public void setUp() { Column.physical("varchar", DataTypes.STRING()), Column.physical("decimal(10, 5)", DataTypes.DECIMAL(10, 5)), Column.physical( - "timestamp", DataTypes.TIMESTAMP(6).bridgedTo(Timestamp.class))); + "timestamp", DataTypes.TIMESTAMP(6).bridgedTo(Timestamp.class)), + Column.physical("binary", DataTypes.BYTES())); rowDataToStringConverter = new RowDataToStringConverterImpl(schema.toPhysicalRowDataType()); List rows = @@ -94,7 +93,8 @@ public void setUp() { 2L, "abc", BigDecimal.valueOf(1.23), - Timestamp.valueOf("2020-03-01 18:39:14")), + Timestamp.valueOf("2020-03-01 18:39:14"), + new byte[] {50, 51, 52, -123, 54, 93, 115, 126}), Row.ofKind( RowKind.UPDATE_BEFORE, false, @@ -102,7 +102,8 @@ public void setUp() { 0L, "", BigDecimal.valueOf(1), - Timestamp.valueOf("2020-03-01 18:39:14.1")), + Timestamp.valueOf("2020-03-01 18:39:14.1"), + new byte[] {100, -98, 32, 121, -125}), Row.ofKind( RowKind.UPDATE_AFTER, true, @@ -110,7 +111,8 @@ public void setUp() { null, "abcdefg", BigDecimal.valueOf(12345), - Timestamp.valueOf("2020-03-01 18:39:14.12")), + Timestamp.valueOf("2020-03-01 18:39:14.12"), + new byte[] {-110, -23, 1, 2}), Row.ofKind( RowKind.DELETE, false, @@ -118,7 +120,8 @@ public void setUp() { Long.MAX_VALUE, null, BigDecimal.valueOf(12345.06789), - Timestamp.valueOf("2020-03-01 18:39:14.123")), + Timestamp.valueOf("2020-03-01 18:39:14.123"), + new byte[] {50, 51, 52, -123, 54, 93, 115, 126}), Row.ofKind( RowKind.INSERT, true, @@ -126,7 +129,8 @@ public void setUp() { Long.MIN_VALUE, "abcdefg111", null, - Timestamp.valueOf("2020-03-01 18:39:14.123456")), + Timestamp.valueOf("2020-03-01 18:39:14.123456"), + new byte[] {110, 23, -1, -2}), Row.ofKind( RowKind.DELETE, null, @@ -134,6 +138,7 @@ public void setUp() { -1L, "abcdefghijklmnopqrstuvwxyz", BigDecimal.valueOf(-12345.06789), + null, null), Row.ofKind( RowKind.INSERT, @@ -142,7 +147,8 @@ public void setUp() { -1L, "这是一段中文", BigDecimal.valueOf(-12345.06789), - Timestamp.valueOf("2020-03-04 18:39:14")), + Timestamp.valueOf("2020-03-04 18:39:14"), + new byte[] {-3, -2, -1, 0, 1, 2, 3}), Row.ofKind( RowKind.DELETE, null, @@ -150,7 +156,8 @@ public void setUp() { -1L, "これは日本語をテストするための文です", BigDecimal.valueOf(-12345.06789), - Timestamp.valueOf("2020-03-04 18:39:14"))); + Timestamp.valueOf("2020-03-04 18:39:14"), + new byte[] {-3, -2, -1, 0, 1, 2, 3})); final DataStructureConverter dataStructureConverter = DataStructureConverters.getConverter(schema.toPhysicalRowDataType()); @@ -189,29 +196,29 @@ public void testBatchResult() { view.displayResults(); view.close(); Assert.assertEquals( - "+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+" + "+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+---------------------+" + System.lineSeparator() - + "| boolean | int | bigint | varchar | decimal(10, 5) | timestamp |" + + "| boolean | int | bigint | varchar | decimal(10, 5) | timestamp | binary |" + System.lineSeparator() - + "+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+" + + "+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+---------------------+" + System.lineSeparator() - + "| | 1 | 2 | abc | 1.23000 | 2020-03-01 18:39:14.000000 |" + + "| | 1 | 2 | abc | 1.23000 | 2020-03-01 18:39:14.000000 | x'32333485365d737e' |" + System.lineSeparator() - + "| FALSE | | 0 | | 1.00000 | 2020-03-01 18:39:14.100000 |" + + "| FALSE | | 0 | | 1.00000 | 2020-03-01 18:39:14.100000 | x'649e207983' |" + System.lineSeparator() - + "| TRUE | 2147483647 | | abcdefg | 12345.00000 | 2020-03-01 18:39:14.120000 |" + + "| TRUE | 2147483647 | | abcdefg | 12345.00000 | 2020-03-01 18:39:14.120000 | x'92e90102' |" + System.lineSeparator() - + "| FALSE | -2147483648 | 9223372036854775807 | | 12345.06789 | 2020-03-01 18:39:14.123000 |" + + "| FALSE | -2147483648 | 9223372036854775807 | | 12345.06789 | 2020-03-01 18:39:14.123000 | x'32333485365d737e' |" + System.lineSeparator() - + "| TRUE | 100 | -9223372036854775808 | abcdefg111 | | 2020-03-01 18:39:14.123456 |" + + "| TRUE | 100 | -9223372036854775808 | abcdefg111 | | 2020-03-01 18:39:14.123456 | x'6e17fffe' |" + System.lineSeparator() - + "| | -1 | -1 | abcdefghijklmnopqrstuvwxyz | -12345.06789 | |" + + "| | -1 | -1 | abcdefghijklmnopqrstuvwxyz | -12345.06789 | | |" + System.lineSeparator() - + "| | -1 | -1 | 这是一段中文 | -12345.06789 | 2020-03-04 18:39:14.000000 |" + + "| | -1 | -1 | 这是一段中文 | -12345.06789 | 2020-03-04 18:39:14.000000 | x'fdfeff00010203' |" + System.lineSeparator() - + "| | -1 | -1 | これは日本語をテストするた... | -12345.06789 | 2020-03-04 18:39:14.000000 |" + + "| | -1 | -1 | これは日本語をテストするた... | -12345.06789 | 2020-03-04 18:39:14.000000 | x'fdfeff00010203' |" + System.lineSeparator() - + "+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+" + + "+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+---------------------+" + System.lineSeparator() + "8 rows in set" + System.lineSeparator(), @@ -243,9 +250,7 @@ public void testCancelBatchResult() throws Exception { // wait until we trying to get batch result CommonTestUtils.waitUntilCondition( - () -> mockExecutor.getNumRetrieveResultChancesCalls() > 1, - Deadline.now().plus(Duration.ofSeconds(5)), - 50L); + () -> mockExecutor.getNumRetrieveResultChancesCalls() > 1, 50L); // send signal to cancel terminal.raise(Terminal.Signal.INT); @@ -354,29 +359,29 @@ public void testStreamingResult() { // source file // by vim or just cat the file to check the regular result. Assert.assertEquals( - "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+" + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+--------------------------------+" + System.lineSeparator() - + "| op | boolean | int | bigint | varchar | decimal(10, 5) | timestamp |" + + "| op | boolean | int | bigint | varchar | decimal(10, 5) | timestamp | binary |" + System.lineSeparator() - + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+" + + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+--------------------------------+" + System.lineSeparator() - + "| +I | | 1 | 2 | abc | 1.23000 | 2020-03-01 18:39:14.000000 |" + + "| +I | | 1 | 2 | abc | 1.23000 | 2020-03-01 18:39:14.000000 | x'32333485365d737e' |" + System.lineSeparator() - + "| -U | FALSE | | 0 | | 1.00000 | 2020-03-01 18:39:14.100000 |" + + "| -U | FALSE | | 0 | | 1.00000 | 2020-03-01 18:39:14.100000 | x'649e207983' |" + System.lineSeparator() - + "| +U | TRUE | 2147483647 | | abcdefg | 12345.00000 | 2020-03-01 18:39:14.120000 |" + + "| +U | TRUE | 2147483647 | | abcdefg | 12345.00000 | 2020-03-01 18:39:14.120000 | x'92e90102' |" + System.lineSeparator() - + "| -D | FALSE | -2147483648 | 9223372036854775807 | | 12345.06789 | 2020-03-01 18:39:14.123000 |" + + "| -D | FALSE | -2147483648 | 9223372036854775807 | | 12345.06789 | 2020-03-01 18:39:14.123000 | x'32333485365d737e' |" + System.lineSeparator() - + "| +I | TRUE | 100 | -9223372036854775808 | abcdefg111 | | 2020-03-01 18:39:14.123456 |" + + "| +I | TRUE | 100 | -9223372036854775808 | abcdefg111 | | 2020-03-01 18:39:14.123456 | x'6e17fffe' |" + System.lineSeparator() - + "| -D | | -1 | -1 | abcdefghijklmnopqrstuvwxyz | -12345.06789 | |" + + "| -D | | -1 | -1 | abcdefghijklmnopqrstuvwxyz | -12345.06789 | | |" + System.lineSeparator() - + "| +I | | -1 | -1 | 这是一段中文 | -12345.06789 | 2020-03-04 18:39:14.000000 |" + + "| +I | | -1 | -1 | 这是一段中文 | -12345.06789 | 2020-03-04 18:39:14.000000 | x'fdfeff00010203' |" + System.lineSeparator() - + "| -D | | -1 | -1 | これは日本語をテストするた... | -12345.06789 | 2020-03-04 18:39:14.000000 |" + + "| -D | | -1 | -1 | これは日本語をテストするた... | -12345.06789 | 2020-03-04 18:39:14.000000 | x'fdfeff00010203' |" + System.lineSeparator() - + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+" + + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+--------------------------------+" + System.lineSeparator() + "Received a total of 8 rows" + System.lineSeparator(), @@ -404,11 +409,11 @@ public void testEmptyStreamingResult() { view.close(); Assert.assertEquals( - "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+" + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+--------------------------------+" + System.lineSeparator() - + "| op | boolean | int | bigint | varchar | decimal(10, 5) | timestamp |" + + "| op | boolean | int | bigint | varchar | decimal(10, 5) | timestamp | binary |" + System.lineSeparator() - + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+" + + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+--------------------------------+" + System.lineSeparator() + "Received a total of 0 row" + System.lineSeparator(), @@ -442,9 +447,7 @@ public void testCancelStreamingResult() throws Exception { // wait until we processed first result CommonTestUtils.waitUntilCondition( - () -> mockExecutor.getNumRetrieveResultChancesCalls() > 1, - Deadline.now().plus(Duration.ofSeconds(5)), - 50L); + () -> mockExecutor.getNumRetrieveResultChancesCalls() > 1, 50L); // send signal to cancel terminal.raise(Terminal.Signal.INT); @@ -452,19 +455,19 @@ public void testCancelStreamingResult() throws Exception { view.close(); Assert.assertEquals( - "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+" + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+--------------------------------+" + System.lineSeparator() - + "| op | boolean | int | bigint | varchar | decimal(10, 5) | timestamp |" + + "| op | boolean | int | bigint | varchar | decimal(10, 5) | timestamp | binary |" + System.lineSeparator() - + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+" + + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+--------------------------------+" + System.lineSeparator() - + "| +I | | 1 | 2 | abc | 1.23000 | 2020-03-01 18:39:14.000000 |" + + "| +I | | 1 | 2 | abc | 1.23000 | 2020-03-01 18:39:14.000000 | x'32333485365d737e' |" + System.lineSeparator() - + "| -U | FALSE | | 0 | | 1.00000 | 2020-03-01 18:39:14.100000 |" + + "| -U | FALSE | | 0 | | 1.00000 | 2020-03-01 18:39:14.100000 | x'649e207983' |" + System.lineSeparator() - + "| +U | TRUE | 2147483647 | | abcdefg | 12345.00000 | 2020-03-01 18:39:14.120000 |" + + "| +U | TRUE | 2147483647 | | abcdefg | 12345.00000 | 2020-03-01 18:39:14.120000 | x'92e90102' |" + System.lineSeparator() - + "| -D | FALSE | -2147483648 | 9223372036854775807 | | 12345.06789 | 2020-03-01 18:39:14.123000 |" + + "| -D | FALSE | -2147483648 | 9223372036854775807 | | 12345.06789 | 2020-03-01 18:39:14.123000 | x'32333485365d737e' |" + System.lineSeparator() + "Query terminated, received a total of 4 rows" + System.lineSeparator(), @@ -504,19 +507,19 @@ public void testFailedStreamingResult() { view.close(); Assert.assertEquals( - "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+" + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+--------------------------------+" + System.lineSeparator() - + "| op | boolean | int | bigint | varchar | decimal(10, 5) | timestamp |" + + "| op | boolean | int | bigint | varchar | decimal(10, 5) | timestamp | binary |" + System.lineSeparator() - + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+" + + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+--------------------------------+" + System.lineSeparator() - + "| +I | | 1 | 2 | abc | 1.23000 | 2020-03-01 18:39:14.000000 |" + + "| +I | | 1 | 2 | abc | 1.23000 | 2020-03-01 18:39:14.000000 | x'32333485365d737e' |" + System.lineSeparator() - + "| -U | FALSE | | 0 | | 1.00000 | 2020-03-01 18:39:14.100000 |" + + "| -U | FALSE | | 0 | | 1.00000 | 2020-03-01 18:39:14.100000 | x'649e207983' |" + System.lineSeparator() - + "| +U | TRUE | 2147483647 | | abcdefg | 12345.00000 | 2020-03-01 18:39:14.120000 |" + + "| +U | TRUE | 2147483647 | | abcdefg | 12345.00000 | 2020-03-01 18:39:14.120000 | x'92e90102' |" + System.lineSeparator() - + "| -D | FALSE | -2147483648 | 9223372036854775807 | | 12345.06789 | 2020-03-01 18:39:14.123000 |" + + "| -D | FALSE | -2147483648 | 9223372036854775807 | | 12345.06789 | 2020-03-01 18:39:14.123000 | x'32333485365d737e' |" + System.lineSeparator(), terminalOutput.toString()); assertThat(mockExecutor.getNumCancelCalls(), is(1)); diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/TestingExecutor.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/TestingExecutor.java index 759db8f79c2d0..fd8fb09df8be4 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/TestingExecutor.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/TestingExecutor.java @@ -19,7 +19,6 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.internal.TableResultInternal; import org.apache.flink.table.client.cli.utils.SqlParserHelper; import org.apache.flink.table.client.gateway.Executor; @@ -40,7 +39,7 @@ /** A customizable {@link Executor} for testing purposes. */ class TestingExecutor implements Executor { - private static final Configuration defaultConfig = TableConfig.getDefault().getConfiguration(); + private static final Configuration defaultConfig = new Configuration(); private int numCancelCalls = 0; private int numRetrieveResultChancesCalls = 0; diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/context/SessionContextTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/context/SessionContextTest.java index 36dadee27ba3a..8078fab0899e1 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/context/SessionContextTest.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/context/SessionContextTest.java @@ -20,6 +20,7 @@ import org.apache.flink.client.cli.DefaultCLI; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.client.gateway.utils.UserDefinedFunctions; import org.apache.flink.table.utils.TestUserClassLoaderJar; @@ -83,19 +84,19 @@ public void testSetAndResetOption() { sessionContext.set(NAME.key(), "test"); // runtime config from flink-conf sessionContext.set(OBJECT_REUSE.key(), "false"); - assertEquals("hive", getConfiguration().getString(TABLE_SQL_DIALECT)); - assertEquals(128, getConfiguration().getInteger(MAX_PARALLELISM)); - assertEquals("test", getConfiguration().getString(NAME)); - assertFalse(getConfiguration().getBoolean(OBJECT_REUSE)); + assertEquals("hive", getConfiguration().get(TABLE_SQL_DIALECT)); + assertEquals(128, (int) getConfiguration().get(MAX_PARALLELISM)); + assertEquals("test", getConfiguration().get(NAME)); + assertFalse(getConfiguration().get(OBJECT_REUSE)); sessionContext.reset(); - assertEquals("default", getConfiguration().getString(TABLE_SQL_DIALECT)); + assertEquals("default", getConfiguration().get(TABLE_SQL_DIALECT)); assertNull(getConfiguration().get(NAME)); // The value of MAX_PARALLELISM in DEFAULTS_ENVIRONMENT_FILE is 16 - assertEquals(16, getConfiguration().getInteger(MAX_PARALLELISM)); - assertNull(getConfiguration().getString(NAME, null)); + assertEquals(16, (int) getConfiguration().get(MAX_PARALLELISM)); + assertNull(getConfiguration().get(NAME)); // The value of OBJECT_REUSE in origin configuration is true - assertTrue(getConfiguration().getBoolean(OBJECT_REUSE)); + assertTrue(getConfiguration().get(OBJECT_REUSE)); } @Test @@ -109,22 +110,22 @@ public void testSetAndResetKeyInConfigOptions() { // runtime config from flink-conf sessionContext.set(OBJECT_REUSE.key(), "false"); - assertEquals("hive", getConfiguration().getString(TABLE_SQL_DIALECT)); - assertEquals(128, getConfiguration().getInteger(MAX_PARALLELISM)); - assertEquals("test", getConfiguration().getString(NAME)); - assertFalse(getConfiguration().getBoolean(OBJECT_REUSE)); + assertEquals("hive", getConfiguration().get(TABLE_SQL_DIALECT)); + assertEquals(128, (int) getConfiguration().get(MAX_PARALLELISM)); + assertEquals("test", getConfiguration().get(NAME)); + assertFalse(getConfiguration().get(OBJECT_REUSE)); sessionContext.reset(TABLE_SQL_DIALECT.key()); - assertEquals("default", getConfiguration().getString(TABLE_SQL_DIALECT)); + assertEquals("default", getConfiguration().get(TABLE_SQL_DIALECT)); sessionContext.reset(MAX_PARALLELISM.key()); - assertEquals(16, getConfiguration().getInteger(MAX_PARALLELISM)); + assertEquals(16, (int) getConfiguration().get(MAX_PARALLELISM)); sessionContext.reset(NAME.key()); assertNull(getConfiguration().get(NAME)); sessionContext.reset(OBJECT_REUSE.key()); - assertTrue(getConfiguration().getBoolean(OBJECT_REUSE)); + assertTrue(getConfiguration().get(OBJECT_REUSE)); } @Test @@ -222,12 +223,8 @@ private Map getConfigurationMap() { .toMap(); } - private Configuration getConfiguration() { - return sessionContext - .getExecutionContext() - .getTableEnvironment() - .getConfig() - .getConfiguration(); + private ReadableConfig getConfiguration() { + return sessionContext.getExecutionContext().getTableEnvironment().getConfig(); } private void validateAddJar(String jarPath) throws IOException { diff --git a/flink-table/flink-sql-client/src/test/resources/sql/set.q b/flink-table/flink-sql-client/src/test/resources/sql/set.q index 5da0b8841c166..5768953d40105 100644 --- a/flink-table/flink-sql-client/src/test/resources/sql/set.q +++ b/flink-table/flink-sql-client/src/test/resources/sql/set.q @@ -16,10 +16,25 @@ # limitations under the License. # test set a configuration +SET 'sql-client.execution.result-mode' = 'tableau'; +[INFO] Session property has been set. +!info + SET 'table.sql-dialect' = 'hive'; [INFO] Session property has been set. !info +create catalog hivecatalog with ( + 'type' = 'hive-test', + 'hive-version' = '2.3.4' +); +[INFO] Execute statement succeed. +!info + +use catalog hivecatalog; +[INFO] Execute statement succeed. +!info + # test create a hive table to verify the configuration works CREATE TABLE hive_table ( product_id STRING, @@ -28,7 +43,7 @@ CREATE TABLE hive_table ( pv_count BIGINT, like_count BIGINT, comment_count BIGINT, - update_time TIMESTAMP(3), + update_time TIMESTAMP, update_user STRING ) PARTITIONED BY (pt_year STRING, pt_month STRING, pt_day STRING) TBLPROPERTIES ( 'streaming-source.enable' = 'true' @@ -36,6 +51,16 @@ CREATE TABLE hive_table ( [INFO] Execute statement succeed. !info +# test "ctas" only supported in Hive Dialect +CREATE TABLE foo as select 1; ++-------------------------+ +| hivecatalog.default.foo | ++-------------------------+ +| -1 | ++-------------------------+ +1 row in set +!ok + # list the configured configuration set; 'execution.attached' = 'true' @@ -47,6 +72,7 @@ set; 'pipeline.classpaths' = '' 'pipeline.jars' = '' 'rest.port' = '$VAR_REST_PORT' +'sql-client.execution.result-mode' = 'tableau' 'table.exec.legacy-cast-behaviour' = 'DISABLED' 'table.sql-dialect' = 'hive' !ok @@ -170,3 +196,11 @@ SELECT id, func1(str) FROM (VALUES (1, 'Hello World')) AS T(id, str) ; +----+-------------+--------------------------------+ Received a total of 1 row !ok + +REMOVE JAR '$VAR_UDF_JAR_PATH'; +[INFO] The specified jar is removed from session classloader. +!info + +SHOW JARS; +Empty set +!ok diff --git a/flink-table/flink-sql-parser-hive/pom.xml b/flink-table/flink-sql-parser-hive/pom.xml index d5459052c923d..8aa0d31ab5702 100644 --- a/flink-table/flink-sql-parser-hive/pom.xml +++ b/flink-table/flink-sql-parser-hive/pom.xml @@ -24,7 +24,7 @@ under the License. flink-table org.apache.flink - 1.15-SNAPSHOT + 1.15.0 flink-sql-parser-hive diff --git a/flink-table/flink-sql-parser/pom.xml b/flink-table/flink-sql-parser/pom.xml index 216408ae1414b..82587b87ba8b8 100644 --- a/flink-table/flink-sql-parser/pom.xml +++ b/flink-table/flink-sql-parser/pom.xml @@ -24,7 +24,7 @@ under the License. flink-table org.apache.flink - 1.15-SNAPSHOT + 1.15.0 flink-sql-parser diff --git a/flink-table/flink-table-api-bridge-base/pom.xml b/flink-table/flink-table-api-bridge-base/pom.xml index 31628f0fb5b75..cca23fe6a52b0 100644 --- a/flink-table/flink-table-api-bridge-base/pom.xml +++ b/flink-table/flink-table-api-bridge-base/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-table-api-bridge-base/src/main/java/org/apache/flink/table/api/bridge/internal/AbstractStreamTableEnvironmentImpl.java b/flink-table/flink-table-api-bridge-base/src/main/java/org/apache/flink/table/api/bridge/internal/AbstractStreamTableEnvironmentImpl.java index c9fb822c66d4c..16c52b77be2f9 100644 --- a/flink-table/flink-table-api-bridge-base/src/main/java/org/apache/flink/table/api/bridge/internal/AbstractStreamTableEnvironmentImpl.java +++ b/flink-table/flink-table-api-bridge-base/src/main/java/org/apache/flink/table/api/bridge/internal/AbstractStreamTableEnvironmentImpl.java @@ -98,14 +98,12 @@ public AbstractStreamTableEnvironmentImpl( } public static Executor lookupExecutor( - ClassLoader classLoader, - String executorIdentifier, - StreamExecutionEnvironment executionEnvironment) { + ClassLoader classLoader, StreamExecutionEnvironment executionEnvironment) { final ExecutorFactory executorFactory; try { executorFactory = FactoryUtil.discoverFactory( - classLoader, ExecutorFactory.class, executorIdentifier); + classLoader, ExecutorFactory.class, ExecutorFactory.DEFAULT_IDENTIFIER); } catch (Exception e) { throw new TableException( "Could not instantiate the executor. Make sure a planner module is on the classpath", @@ -227,7 +225,8 @@ protected DataStream toStreamInternal(Table table, ModifyOperation modify final Transformation transformation = getTransformation(table, transformations); executionEnvironment.addOperator(transformation); - // reconfigure whenever planner transformations are added + // Reconfigure whenever planner transformations are added + // We pass only the configuration to avoid reconfiguration with the rootConfiguration executionEnvironment.configure(tableConfig.getConfiguration()); return new DataStream<>(executionEnvironment, transformation); diff --git a/flink-table/flink-table-api-java-bridge/pom.xml b/flink-table/flink-table-api-java-bridge/pom.xml index c2246fe10124a..094bf2f48b8f6 100644 --- a/flink-table/flink-table-api-java-bridge/pom.xml +++ b/flink-table/flink-table-api-java-bridge/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/StreamTableEnvironment.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/StreamTableEnvironment.java index 6481acff4504a..6c968582294f7 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/StreamTableEnvironment.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/StreamTableEnvironment.java @@ -31,7 +31,6 @@ import org.apache.flink.table.api.Schema; import org.apache.flink.table.api.StatementSet; import org.apache.flink.table.api.Table; -import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl; import org.apache.flink.table.connector.ChangelogMode; @@ -92,9 +91,7 @@ public interface StreamTableEnvironment extends TableEnvironment { * TableEnvironment}. */ static StreamTableEnvironment create(StreamExecutionEnvironment executionEnvironment) { - return create( - executionEnvironment, - EnvironmentSettings.fromConfiguration(executionEnvironment.getConfiguration())); + return create(executionEnvironment, EnvironmentSettings.newInstance().build()); } /** @@ -122,43 +119,7 @@ static StreamTableEnvironment create(StreamExecutionEnvironment executionEnviron */ static StreamTableEnvironment create( StreamExecutionEnvironment executionEnvironment, EnvironmentSettings settings) { - TableConfig tableConfig = new TableConfig(); - tableConfig.addConfiguration(settings.toConfiguration()); - return StreamTableEnvironmentImpl.create(executionEnvironment, settings, tableConfig); - } - - /** - * Creates a table environment that is the entry point and central context for creating Table - * and SQL API programs that integrate with the Java-specific {@link DataStream} API. - * - *

    It is unified for bounded and unbounded data processing. - * - *

    A stream table environment is responsible for: - * - *

      - *
    • Convert a {@link DataStream} into {@link Table} and vice-versa. - *
    • Connecting to external systems. - *
    • Registering and retrieving {@link Table}s and other meta objects from a catalog. - *
    • Executing SQL statements. - *
    • Offering further configuration options. - *
    - * - *

    Note: If you don't intend to use the {@link DataStream} API, {@link TableEnvironment} is - * meant for pure table programs. - * - * @param executionEnvironment The Java {@link StreamExecutionEnvironment} of the {@link - * TableEnvironment}. - * @param tableConfig The configuration of the {@link TableEnvironment}. - * @deprecated Use {@link #create(StreamExecutionEnvironment)} and {@link #getConfig()} for - * manipulating {@link TableConfig}. - */ - @Deprecated - static StreamTableEnvironment create( - StreamExecutionEnvironment executionEnvironment, TableConfig tableConfig) { - return StreamTableEnvironmentImpl.create( - executionEnvironment, - EnvironmentSettings.fromConfiguration(tableConfig.getConfiguration()), - tableConfig); + return StreamTableEnvironmentImpl.create(executionEnvironment, settings); } /** diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/internal/StreamTableEnvironmentImpl.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/internal/StreamTableEnvironmentImpl.java index c6c7d43e222aa..cb8d38b8c6b41 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/internal/StreamTableEnvironmentImpl.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/internal/StreamTableEnvironmentImpl.java @@ -92,19 +92,23 @@ public StreamTableEnvironmentImpl( } public static StreamTableEnvironment create( - StreamExecutionEnvironment executionEnvironment, - EnvironmentSettings settings, - TableConfig tableConfig) { + StreamExecutionEnvironment executionEnvironment, EnvironmentSettings settings) { // temporary solution until FLINK-15635 is fixed final ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); + final Executor executor = lookupExecutor(classLoader, executionEnvironment); + + final TableConfig tableConfig = TableConfig.getDefault(); + tableConfig.setRootConfiguration(executor.getConfiguration()); + tableConfig.addConfiguration(settings.getConfiguration()); + final ModuleManager moduleManager = new ModuleManager(); final CatalogManager catalogManager = CatalogManager.newBuilder() .classLoader(classLoader) - .config(tableConfig.getConfiguration()) + .config(tableConfig) .defaultCatalog( settings.getBuiltInCatalogName(), new GenericInMemoryCatalog( @@ -116,17 +120,9 @@ public static StreamTableEnvironment create( final FunctionCatalog functionCatalog = new FunctionCatalog(tableConfig, catalogManager, moduleManager); - final Executor executor = - lookupExecutor(classLoader, settings.getExecutor(), executionEnvironment); - final Planner planner = PlannerFactoryUtil.createPlanner( - settings.getPlanner(), - executor, - tableConfig, - moduleManager, - catalogManager, - functionCatalog); + executor, tableConfig, moduleManager, catalogManager, functionCatalog); return new StreamTableEnvironmentImpl( catalogManager, diff --git a/flink-table/flink-table-api-java-uber/pom.xml b/flink-table/flink-table-api-java-uber/pom.xml index e0f68eb3db000..35e67d9517ac1 100644 --- a/flink-table/flink-table-api-java-uber/pom.xml +++ b/flink-table/flink-table-api-java-uber/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-table-api-java/pom.xml b/flink-table/flink-table-api-java/pom.xml index ba16fbf01215c..8a492592bb05a 100644 --- a/flink-table/flink-table-api-java/pom.xml +++ b/flink-table/flink-table-api-java/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/EnvironmentSettings.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/EnvironmentSettings.java index cc9a32b703be5..258fcb8acdb64 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/EnvironmentSettings.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/EnvironmentSettings.java @@ -18,21 +18,17 @@ package org.apache.flink.table.api; -import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.delegation.Executor; -import org.apache.flink.table.delegation.ExecutorFactory; -import org.apache.flink.table.delegation.Planner; -import org.apache.flink.table.delegation.PlannerFactory; +import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.functions.UserDefinedFunction; -import javax.annotation.Nullable; - import static org.apache.flink.api.common.RuntimeExecutionMode.BATCH; import static org.apache.flink.api.common.RuntimeExecutionMode.STREAMING; import static org.apache.flink.configuration.ExecutionOptions.RUNTIME_MODE; +import static org.apache.flink.table.api.config.TableConfigOptions.TABLE_CATALOG_NAME; +import static org.apache.flink.table.api.config.TableConfigOptions.TABLE_DATABASE_NAME; /** * Defines all parameters that initialize a table environment. Those parameters are used only during @@ -60,44 +56,14 @@ public class EnvironmentSettings { private static final EnvironmentSettings DEFAULT_BATCH_MODE_SETTINGS = EnvironmentSettings.newInstance().inBatchMode().build(); - public static final String DEFAULT_BUILTIN_CATALOG = "default_catalog"; - public static final String DEFAULT_BUILTIN_DATABASE = "default_database"; - - /** Factory identifier of the {@link Planner} to use. */ - private final String planner; - - /** Factory identifier of the {@link Executor} to use. */ - private final String executor; - - /** - * Specifies the name of the initial catalog to be created when instantiating {@link - * TableEnvironment}. - */ - private final String builtInCatalogName; - /** - * Specifies the name of the default database in the initial catalog to be created when - * instantiating {@link TableEnvironment}. + * Holds all the configuration generated by the builder, together with any required additional + * configuration. */ - private final String builtInDatabaseName; + private final Configuration configuration; - /** - * Determines if the table environment should work in a batch ({@code false}) or streaming - * ({@code true}) mode. - */ - private final boolean isStreamingMode; - - private EnvironmentSettings( - String planner, - @Nullable String executor, - String builtInCatalogName, - String builtInDatabaseName, - boolean isStreamingMode) { - this.planner = planner; - this.executor = executor; - this.builtInCatalogName = builtInCatalogName; - this.builtInDatabaseName = builtInDatabaseName; - this.isStreamingMode = isStreamingMode; + private EnvironmentSettings(Configuration configuration) { + this.configuration = configuration; } /** @@ -130,32 +96,28 @@ public static Builder newInstance() { return new Builder(); } - /** Creates an instance of {@link EnvironmentSettings} from configuration. */ + /** + * Creates an instance of {@link EnvironmentSettings} from configuration. + * + * @deprecated use {@link Builder#withConfiguration(Configuration)} instead. + */ + @Deprecated public static EnvironmentSettings fromConfiguration(ReadableConfig configuration) { - final Builder builder = new Builder(); - switch (configuration.get(RUNTIME_MODE)) { - case STREAMING: - builder.inStreamingMode(); - break; - case BATCH: - builder.inBatchMode(); - break; - case AUTOMATIC: - default: - throw new TableException( - String.format( - "Unsupported mode '%s' for '%s'. " - + "Only an explicit BATCH or STREAMING mode is supported in Table API.", - configuration.get(RUNTIME_MODE), RUNTIME_MODE.key())); - } - - return builder.build(); + return new EnvironmentSettings((Configuration) configuration); } - /** Convert the environment setting to the {@link Configuration}. */ + /** + * Convert the environment setting to the {@link Configuration}. + * + * @deprecated use {@link #getConfiguration} instead. + */ + @Deprecated public Configuration toConfiguration() { - Configuration configuration = new Configuration(); - configuration.set(RUNTIME_MODE, isStreamingMode() ? STREAMING : BATCH); + return configuration; + } + + /** Get the underlying {@link Configuration}. */ + public Configuration getConfiguration() { return configuration; } @@ -164,7 +126,7 @@ public Configuration toConfiguration() { * TableEnvironment}. */ public String getBuiltInCatalogName() { - return builtInCatalogName; + return configuration.get(TABLE_CATALOG_NAME); } /** @@ -172,45 +134,31 @@ public String getBuiltInCatalogName() { * instantiating a {@link TableEnvironment}. */ public String getBuiltInDatabaseName() { - return builtInDatabaseName; + return configuration.get(TABLE_DATABASE_NAME); } /** Tells if the {@link TableEnvironment} should work in a batch or streaming mode. */ public boolean isStreamingMode() { - return isStreamingMode; - } - - /** Returns the identifier of the {@link Planner} to be used. */ - @Internal - public String getPlanner() { - return planner; - } - - /** Returns the {@link Executor} that should submit and execute table programs. */ - @Internal - public String getExecutor() { - return executor; + return configuration.get(RUNTIME_MODE) == STREAMING; } /** A builder for {@link EnvironmentSettings}. */ @PublicEvolving public static class Builder { - private final String planner = PlannerFactory.DEFAULT_IDENTIFIER; - private final String executor = ExecutorFactory.DEFAULT_IDENTIFIER; - private String builtInCatalogName = DEFAULT_BUILTIN_CATALOG; - private String builtInDatabaseName = DEFAULT_BUILTIN_DATABASE; - private boolean isStreamingMode = true; + private final Configuration configuration = new Configuration(); + + public Builder() {} /** Sets that the components should work in a batch mode. Streaming mode by default. */ public Builder inBatchMode() { - this.isStreamingMode = false; + configuration.set(RUNTIME_MODE, BATCH); return this; } /** Sets that the components should work in a streaming mode. Enabled by default. */ public Builder inStreamingMode() { - this.isStreamingMode = true; + configuration.set(RUNTIME_MODE, STREAMING); return this; } @@ -226,10 +174,10 @@ public Builder inStreamingMode() { *

    It will also be the initial value for the current catalog which can be altered via * {@link TableEnvironment#useCatalog(String)}. * - *

    Default: "default_catalog". + *

    Default: {@link TableConfigOptions#TABLE_DATABASE_NAME}{@code .defaultValue()}. */ public Builder withBuiltInCatalogName(String builtInCatalogName) { - this.builtInCatalogName = builtInCatalogName; + configuration.set(TABLE_CATALOG_NAME, builtInCatalogName); return this; } @@ -245,17 +193,22 @@ public Builder withBuiltInCatalogName(String builtInCatalogName) { *

    It will also be the initial value for the current database which can be altered via * {@link TableEnvironment#useDatabase(String)}. * - *

    Default: "default_database". + *

    Default: {@link TableConfigOptions#TABLE_DATABASE_NAME}{@code .defaultValue()}. */ public Builder withBuiltInDatabaseName(String builtInDatabaseName) { - this.builtInDatabaseName = builtInDatabaseName; + configuration.set(TABLE_DATABASE_NAME, builtInDatabaseName); + return this; + } + + /** Add extra configuration to {@link EnvironmentSettings}. */ + public Builder withConfiguration(Configuration configuration) { + this.configuration.addAll(configuration); return this; } /** Returns an immutable instance of {@link EnvironmentSettings}. */ public EnvironmentSettings build() { - return new EnvironmentSettings( - planner, executor, builtInCatalogName, builtInDatabaseName, isStreamingMode); + return new EnvironmentSettings(configuration); } } } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java index 2096f296286cf..b102fbcc91582 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java @@ -444,7 +444,7 @@ public static ApiExpression rand(Object seed) { } /** - * Returns a pseudorandom integer value between 0.0 (inclusive) and the specified value + * Returns a pseudorandom integer value between 0 (inclusive) and the specified value * (exclusive). */ public static ApiExpression randInteger(Object bound) { @@ -452,7 +452,7 @@ public static ApiExpression randInteger(Object bound) { } /** - * Returns a pseudorandom integer value between 0.0 (inclusive) and the specified value + * Returns a pseudorandom integer value between 0 (inclusive) and the specified value * (exclusive) with a initial seed. Two randInteger() functions will return identical sequences * of numbers if they have same initial seed and same bound. */ diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java index 2565bb4b61d49..823aabe946849 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java @@ -19,15 +19,18 @@ package org.apache.flink.table.api; import org.apache.flink.annotation.Experimental; +import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.PipelineOptions; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.configuration.WritableConfig; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.api.config.OptimizerConfigOptions; import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.table.delegation.Executor; import org.apache.flink.util.Preconditions; import java.math.MathContext; @@ -35,6 +38,7 @@ import java.time.ZoneId; import java.util.HashMap; import java.util.Map; +import java.util.Optional; import static java.time.ZoneId.SHORT_IDS; @@ -42,6 +46,29 @@ * Configuration for the current {@link TableEnvironment} session to adjust Table & SQL API * programs. * + *

    This class is a pure API class that abstracts configuration from various sources. Currently, + * configuration can be set in any of the following layers (in the given order): + * + *

      + *
    1. {@code flink-conf.yaml}, + *
    2. CLI parameters, + *
    3. {@code StreamExecutionEnvironment} when bridging to DataStream API, + *
    4. {@link EnvironmentSettings.Builder#withConfiguration(Configuration)} / {@link + * TableEnvironment#create(Configuration)}, + *
    5. and {@link TableConfig#set(ConfigOption, Object)} / {@link TableConfig#set(String, + * String)}. + *
    + * + *

    The latter two represent the application-specific part of the configuration. They initialize + * and directly modify {@link TableConfig#getConfiguration()}. Other layers represent the + * configuration of the execution context and are immutable. + * + *

    The getters {@link #get(ConfigOption)} and {@link #getOptional(ConfigOption)} give read-only + * access to the full configuration. However, application-specific configuration has precedence. + * Configuration of outer layers is used for defaults and fallbacks. The setters {@link + * #set(ConfigOption, Object)} and {@link #set(String, String)} will only affect + * application-specific configuration. + * *

    For common or important configuration options, this class provides getters and setters methods * with detailed inline documentation. * @@ -68,9 +95,30 @@ * @see OptimizerConfigOptions */ @PublicEvolving -public class TableConfig implements WritableConfig { - /** Defines if all fields need to be checked for NULL first. */ - private Boolean nullCheck = true; +public final class TableConfig implements WritableConfig, ReadableConfig { + + /** Please use {@link TableConfig#getDefault()} instead. */ + @Deprecated + public TableConfig() {} + + // Note to implementers: + // TableConfig is a ReadableConfig which is built once the TableEnvironment is created and + // contains both the configuration defined in the execution context (flink-conf.yaml + CLI + // params), stored in rootConfiguration, but also any extra configuration defined by the user in + // the application, which has precedence over the execution configuration. + // + // This way, any consumer of TableConfig can get the complete view of the configuration + // (environment + user-defined/application-specific) by calling the get() and getOptional() + // methods. + // + // The set() methods only impact the application-specific configuration. + + /** + * Defines if all fields need to be checked for NULL first. + * + * @deprecated This option is not used anymore and will be removed in next releases. + */ + @Deprecated private Boolean nullCheck = true; /** Defines the configuration of Planner for Table API and SQL queries. */ private PlannerConfig plannerConfig = PlannerConfig.EMPTY_CONFIG; @@ -78,14 +126,22 @@ public class TableConfig implements WritableConfig { /** * Defines the default context for decimal division calculation. We use Scala's default * MathContext.DECIMAL128. + * + * @deprecated This option is not used anymore and will be removed in next releases. */ - private MathContext decimalContext = MathContext.DECIMAL128; + @Deprecated private MathContext decimalContext = MathContext.DECIMAL128; - /** A configuration object to hold all key/value configuration. */ + /** + * A configuration object to hold all configuration that has been set specifically in the Table + * API. It does not contain configuration from outer layers. + */ private final Configuration configuration = new Configuration(); + /** Configuration adopted from the outer layer (i.e. the {@link Executor}). */ + private ReadableConfig rootConfiguration = new Configuration(); + /** - * Sets a value for the given {@link ConfigOption}. + * Sets an application-specific value for the given {@link ConfigOption}. * *

    This method should be preferred over {@link #set(String, String)} as it is type-safe, * avoids unnecessary parsing of the value, and provides inline documentation. @@ -104,7 +160,7 @@ public TableConfig set(ConfigOption option, T value) { } /** - * Sets a string-based value for the given string-based key. + * Sets an application-specific string-based value for the given string-based key. * *

    The value will be parsed by the framework on access. * @@ -121,14 +177,62 @@ public TableConfig set(String key, String value) { return this; } - /** Gives direct access to the underlying key-value map for advanced configuration. */ + /** + * {@inheritDoc} + * + *

    This method gives read-only access to the full configuration. However, + * application-specific configuration has precedence. Configuration of outer layers is used for + * defaults and fallbacks. See the docs of {@link TableConfig} for more information. + * + * @param option metadata of the option to read + * @param type of the value to read + * @return read value or {@link ConfigOption#defaultValue()} if not found + */ + @Override + public T get(ConfigOption option) { + return configuration.getOptional(option).orElseGet(() -> rootConfiguration.get(option)); + } + + /** + * {@inheritDoc} + * + *

    This method gives read-only access to the full configuration. However, + * application-specific configuration has precedence. Configuration of outer layers is used for + * defaults and fallbacks. See the docs of {@link TableConfig} for more information. + * + * @param option metadata of the option to read + * @param type of the value to read + * @return read value or {@link Optional#empty()} if not found + */ + @Override + public Optional getOptional(ConfigOption option) { + final Optional tableValue = configuration.getOptional(option); + if (tableValue.isPresent()) { + return tableValue; + } + return rootConfiguration.getOptional(option); + } + + /** + * Gives direct access to the underlying application-specific key-value map for advanced + * configuration. + */ public Configuration getConfiguration() { return configuration; } /** - * Adds the given key-value configuration to the underlying configuration. It overwrites - * existing keys. + * Gives direct access to the underlying environment-specific key-value map for advanced + * configuration. + */ + @Internal + public ReadableConfig getRootConfiguration() { + return rootConfiguration; + } + + /** + * Adds the given key-value configuration to the underlying application-specific configuration. + * It overwrites existing keys. * * @param configuration key-value configuration to be added */ @@ -139,8 +243,7 @@ public void addConfiguration(Configuration configuration) { /** Returns the current SQL dialect. */ public SqlDialect getSqlDialect() { - return SqlDialect.valueOf( - getConfiguration().getString(TableConfigOptions.TABLE_SQL_DIALECT).toUpperCase()); + return SqlDialect.valueOf(get(TableConfigOptions.TABLE_SQL_DIALECT).toUpperCase()); } /** Sets the current SQL dialect to parse a SQL query. Flink's SQL behavior by default. */ @@ -228,12 +331,21 @@ private void validateTimeZone(String zone) { } } - /** Returns the NULL check. If enabled, all fields need to be checked for NULL first. */ + /** + * Returns the NULL check. If enabled, all fields need to be checked for NULL first. + * + * @deprecated This option is not used anymore and will be removed in next releases. + */ + @Deprecated public Boolean getNullCheck() { return nullCheck; } - /** Sets the NULL check. If enabled, all fields need to be checked for NULL first. */ + /** + * Sets the NULL check. If enabled, all fields need to be checked for NULL first. + * + * @deprecated This option is not used anymore and will be removed in next releases. + */ public void setNullCheck(Boolean nullCheck) { this.nullCheck = Preconditions.checkNotNull(nullCheck); } @@ -254,7 +366,10 @@ public void setPlannerConfig(PlannerConfig plannerConfig) { /** * Returns the default context for decimal division calculation. {@link * java.math.MathContext#DECIMAL128} by default. + * + * @deprecated This option is not used anymore and will be removed in next releases. */ + @Deprecated public MathContext getDecimalContext() { return decimalContext; } @@ -262,7 +377,10 @@ public MathContext getDecimalContext() { /** * Sets the default context for decimal division calculation. {@link * java.math.MathContext#DECIMAL128} by default. + * + * @deprecated This option is not used anymore and will be removed in next releases. */ + @Deprecated public void setDecimalContext(MathContext decimalContext) { this.decimalContext = Preconditions.checkNotNull(decimalContext); } @@ -320,9 +438,9 @@ public void setIdleStateRetentionTime(Time minTime, Time maxTime) { && !(maxTime.toMilliseconds() == 0 && minTime.toMilliseconds() == 0)) { throw new IllegalArgumentException( "Difference between minTime: " - + minTime.toString() + + minTime + " and maxTime: " - + maxTime.toString() + + maxTime + " should be at least 5 minutes."); } setIdleStateRetention(Duration.ofMillis(minTime.toMilliseconds())); @@ -398,14 +516,24 @@ public Duration getIdleStateRetention() { @Experimental public void addJobParameter(String key, String value) { Map params = - getConfiguration() - .getOptional(PipelineOptions.GLOBAL_JOB_PARAMETERS) + getOptional(PipelineOptions.GLOBAL_JOB_PARAMETERS) .map(HashMap::new) .orElseGet(HashMap::new); params.put(key, value); set(PipelineOptions.GLOBAL_JOB_PARAMETERS, params); } + /** + * Sets the given configuration as {@link #rootConfiguration}, which contains any configuration + * set in the execution context. See the docs of {@link TableConfig} for more information. + * + * @param rootConfiguration root configuration to be set + */ + @Internal + public void setRootConfiguration(ReadableConfig rootConfiguration) { + this.rootConfiguration = rootConfiguration; + } + public static TableConfig getDefault() { return new TableConfig(); } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableResult.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableResult.java index bf70126db28a6..bb0e0ac1efcb2 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableResult.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableResult.java @@ -178,7 +178,7 @@ default TableSchema getTableSchema() { * *

    This method has slightly different behaviors under different checkpointing settings (to * enable checkpointing for a streaming job, set checkpointing properties through {@link - * TableConfig#getConfiguration()}). + * TableConfig}). * *

      *
    • For batch jobs or streaming jobs without checkpointing, this method has neither @@ -205,7 +205,7 @@ default TableSchema getTableSchema() { * *

      This method has slightly different behaviors under different checkpointing settings (to * enable checkpointing for a streaming job, set checkpointing properties through {@link - * TableConfig#getConfiguration()}). + * TableConfig}). * *

        *
      • For batch jobs or streaming jobs without checkpointing, this method has neither 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 9a92e48b2054c..9aa8642dee4ab 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 @@ -420,7 +420,7 @@ public class ExecutionConfigOptions { public static final ConfigOption TABLE_EXEC_LEGACY_CAST_BEHAVIOUR = key("table.exec.legacy-cast-behaviour") .enumType(LegacyCastBehaviour.class) - .defaultValue(LegacyCastBehaviour.ENABLED) + .defaultValue(LegacyCastBehaviour.DISABLED) .withDescription( "Determines whether CAST will operate following the legacy behaviour " + "or the new one that introduces various fixes and improvements."); diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/TableConfigOptions.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/TableConfigOptions.java index 8601f38bb85c3..d4afa20f7939f 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/TableConfigOptions.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/TableConfigOptions.java @@ -40,6 +40,24 @@ public class TableConfigOptions { private TableConfigOptions() {} + @Documentation.TableOption(execMode = Documentation.ExecMode.BATCH_STREAMING) + public static final ConfigOption TABLE_CATALOG_NAME = + key("table.builtin-catalog-name") + .stringType() + .defaultValue("default_catalog") + .withDescription( + "The name of the initial catalog to be created when " + + "instantiating a TableEnvironment."); + + @Documentation.TableOption(execMode = Documentation.ExecMode.BATCH_STREAMING) + public static final ConfigOption TABLE_DATABASE_NAME = + key("table.builtin-database-name") + .stringType() + .defaultValue("default_database") + .withDescription( + "The name of the default database in the initial catalog to be created " + + "when instantiating TableEnvironment."); + @Documentation.TableOption(execMode = Documentation.ExecMode.BATCH_STREAMING) public static final ConfigOption TABLE_DML_SYNC = key("table.dml-sync") diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/CompiledPlanImpl.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/CompiledPlanImpl.java index 027f8daad425a..6efa375eaf745 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/CompiledPlanImpl.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/CompiledPlanImpl.java @@ -54,10 +54,7 @@ public void writeToFile(File file, boolean ignoreIfExists) { internalPlan.writeToFile( file, ignoreIfExists, - !tableEnvironment - .getConfig() - .getConfiguration() - .get(TableConfigOptions.PLAN_FORCE_RECOMPILE)); + !tableEnvironment.getConfig().get(TableConfigOptions.PLAN_FORCE_RECOMPILE)); } @Override diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java index ea17a9d5e0960..0d5e8c6cea414 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java @@ -265,28 +265,29 @@ protected TableEnvironmentImpl( } public static TableEnvironmentImpl create(Configuration configuration) { - return create(EnvironmentSettings.fromConfiguration(configuration), configuration); + return create(EnvironmentSettings.newInstance().withConfiguration(configuration).build()); } public static TableEnvironmentImpl create(EnvironmentSettings settings) { - return create(settings, settings.toConfiguration()); - } - - private static TableEnvironmentImpl create( - EnvironmentSettings settings, Configuration configuration) { // temporary solution until FLINK-15635 is fixed final ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); + final ExecutorFactory executorFactory = + FactoryUtil.discoverFactory( + classLoader, ExecutorFactory.class, ExecutorFactory.DEFAULT_IDENTIFIER); + final Executor executor = executorFactory.create(settings.getConfiguration()); + // use configuration to init table config final TableConfig tableConfig = new TableConfig(); - tableConfig.addConfiguration(configuration); + tableConfig.setRootConfiguration(executor.getConfiguration()); + tableConfig.addConfiguration(settings.getConfiguration()); final ModuleManager moduleManager = new ModuleManager(); final CatalogManager catalogManager = CatalogManager.newBuilder() .classLoader(classLoader) - .config(tableConfig.getConfiguration()) + .config(tableConfig) .defaultCatalog( settings.getBuiltInCatalogName(), new GenericInMemoryCatalog( @@ -297,19 +298,9 @@ private static TableEnvironmentImpl create( final FunctionCatalog functionCatalog = new FunctionCatalog(tableConfig, catalogManager, moduleManager); - final ExecutorFactory executorFactory = - FactoryUtil.discoverFactory( - classLoader, ExecutorFactory.class, settings.getExecutor()); - final Executor executor = executorFactory.create(configuration); - final Planner planner = PlannerFactoryUtil.createPlanner( - settings.getPlanner(), - executor, - tableConfig, - moduleManager, - catalogManager, - functionCatalog); + executor, tableConfig, moduleManager, catalogManager, functionCatalog); return new TableEnvironmentImpl( catalogManager, @@ -754,7 +745,7 @@ private CompiledPlan compilePlanAndWrite( return loadPlan(PlanReference.fromFile(filePath)); } - if (!tableConfig.getConfiguration().get(TableConfigOptions.PLAN_FORCE_RECOMPILE)) { + if (!tableConfig.get(TableConfigOptions.PLAN_FORCE_RECOMPILE)) { throw new TableException( String.format( "Cannot overwrite the plan file '%s'. " @@ -791,7 +782,7 @@ public TableResultInternal executeInternal(List operations) { List> transformations = translate(operations); List sinkIdentifierNames = extractSinkIdentifierNames(operations); TableResultInternal result = executeInternal(transformations, sinkIdentifierNames); - if (tableConfig.getConfiguration().get(TABLE_DML_SYNC)) { + if (tableConfig.get(TABLE_DML_SYNC)) { try { result.await(); } catch (InterruptedException | ExecutionException e) { @@ -805,6 +796,7 @@ public TableResultInternal executeInternal(List operations) { private TableResultInternal executeInternal( List> transformations, List sinkIdentifierNames) { final String defaultJobName = "insert-into_" + String.join(",", sinkIdentifierNames); + // We pass only the configuration to avoid reconfiguration with the rootConfiguration Pipeline pipeline = execEnv.createPipeline( transformations, tableConfig.getConfiguration(), defaultJobName); @@ -835,6 +827,7 @@ private TableResultInternal executeQueryOperation(QueryOperation operation) { List> transformations = translate(Collections.singletonList(sinkOperation)); final String defaultJobName = "collect"; + // We pass only the configuration to avoid reconfiguration with the rootConfiguration Pipeline pipeline = execEnv.createPipeline( transformations, tableConfig.getConfiguration(), defaultJobName); @@ -1361,10 +1354,7 @@ private TableResultInternal createCatalog(CreateCatalogOperation operation) { Catalog catalog = FactoryUtil.createCatalog( - catalogName, - properties, - tableConfig.getConfiguration(), - userClassLoader); + catalogName, properties, tableConfig, userClassLoader); catalogManager.registerCatalog(catalogName, catalog); return TableResultImpl.TABLE_RESULT_OK; @@ -1380,7 +1370,7 @@ private TableResultInternal loadModule(LoadModuleOperation operation) { FactoryUtil.createModule( operation.getModuleName(), operation.getOptions(), - tableConfig.getConfiguration(), + tableConfig, userClassLoader); moduleManager.loadModule(operation.getModuleName(), module); return TableResultImpl.TABLE_RESULT_OK; diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/DefaultSchemaResolver.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/DefaultSchemaResolver.java index 0b68aeff79b52..f2f011e22c297 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/DefaultSchemaResolver.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/DefaultSchemaResolver.java @@ -43,6 +43,7 @@ import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -95,6 +96,7 @@ public ResolvedSchema resolve(Schema schema) { private List resolveColumns(List unresolvedColumns) { validateDuplicateColumns(unresolvedColumns); + validateDuplicateMetadataKeys(unresolvedColumns); final Column[] resolvedColumns = new Column[unresolvedColumns.size()]; // process source columns first before computed columns @@ -175,6 +177,32 @@ private void validateDuplicateColumns(List columns) { } } + private void validateDuplicateMetadataKeys(List columns) { + Map metadataKeyToColumnNames = new HashMap<>(); + for (Schema.UnresolvedColumn column : columns) { + if (!(column instanceof UnresolvedMetadataColumn)) { + continue; + } + + UnresolvedMetadataColumn metadataColumn = (UnresolvedMetadataColumn) column; + String metadataKey = + metadataColumn.getMetadataKey() == null + ? metadataColumn.getName() + : metadataColumn.getMetadataKey(); + if (metadataKeyToColumnNames.containsKey(metadataKey)) { + throw new ValidationException( + String.format( + "The column `%s` and `%s` in the table are both from the same metadata key '%s'. " + + "Please specify one of the columns as the metadata column and use the " + + "computed column syntax to specify the others.", + metadataKeyToColumnNames.get(metadataKey), + metadataColumn.getName(), + metadataKey)); + } + metadataKeyToColumnNames.put(metadataKey, metadataColumn.getName()); + } + } + private List resolveWatermarkSpecs( List unresolvedWatermarkSpecs, List inputColumns) { if (unresolvedWatermarkSpecs.size() == 0) { diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalog.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalog.java index afa626b05aae1..6b11ec912730b 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalog.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalog.java @@ -21,7 +21,6 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; @@ -76,11 +75,6 @@ public final class FunctionCatalog { */ private PlannerTypeInferenceUtil plannerTypeInferenceUtil; - public FunctionCatalog( - TableConfig tableConfig, CatalogManager catalogManager, ModuleManager moduleManager) { - this(checkNotNull(tableConfig).getConfiguration(), catalogManager, moduleManager); - } - public FunctionCatalog( ReadableConfig config, CatalogManager catalogManager, ModuleManager moduleManager) { this.config = checkNotNull(config); diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/ManagedTableListener.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/ManagedTableListener.java index d2598f406d7eb..80f7c95bbd324 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/ManagedTableListener.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/ManagedTableListener.java @@ -20,13 +20,14 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ExecutionOptions; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.descriptors.ConnectorDescriptorValidator; import org.apache.flink.table.factories.DynamicTableFactory; import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.TableFactoryUtil; import org.apache.flink.util.StringUtils; import javax.annotation.Nullable; @@ -110,7 +111,8 @@ public static boolean isManagedTable(@Nullable Catalog catalog, CatalogBaseTable return false; } - if (table.getTableKind() == CatalogBaseTable.TableKind.VIEW) { + if (table.getTableKind() != CatalogBaseTable.TableKind.TABLE + || !(table instanceof CatalogTable)) { // view is not managed table return false; } @@ -123,8 +125,14 @@ public static boolean isManagedTable(@Nullable Catalog catalog, CatalogBaseTable return false; } - if (!StringUtils.isNullOrWhitespaceOnly( - options.get(ConnectorDescriptorValidator.CONNECTOR_TYPE))) { + // check legacy connector, here we need to check the factory, other properties are dummy + if (TableFactoryUtil.isLegacyConnectorOptions( + catalog, + new Configuration(), + true, + ObjectIdentifier.of("dummy_catalog", "dummy_database", "dummy_table"), + (CatalogTable) table, + true)) { // legacy connector is not managed table return false; } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/ExpressionResolver.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/ExpressionResolver.java index 281866fa4bb1c..ea6f8e35e90c2 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/ExpressionResolver.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/ExpressionResolver.java @@ -137,7 +137,7 @@ private ExpressionResolver( List localReferences, @Nullable DataType outputDataType, boolean isGroupedAggregation) { - this.config = Preconditions.checkNotNull(tableConfig).getConfiguration(); + this.config = Preconditions.checkNotNull(tableConfig); this.tableLookup = Preconditions.checkNotNull(tableLookup); this.fieldLookup = Preconditions.checkNotNull(fieldLookup); this.functionLookup = Preconditions.checkNotNull(functionLookup); diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/factories/PlannerFactoryUtil.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/factories/PlannerFactoryUtil.java index ef64f32bdb984..674c916fc486a 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/factories/PlannerFactoryUtil.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/factories/PlannerFactoryUtil.java @@ -35,7 +35,6 @@ public class PlannerFactoryUtil { /** Discovers a planner factory and creates a planner instance. */ public static Planner createPlanner( - String plannerIdentifier, Executor executor, TableConfig tableConfig, ModuleManager moduleManager, @@ -45,7 +44,7 @@ public static Planner createPlanner( FactoryUtil.discoverFactory( Thread.currentThread().getContextClassLoader(), PlannerFactory.class, - plannerIdentifier); + PlannerFactory.DEFAULT_IDENTIFIER); final Context context = new DefaultPlannerContext( diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/factories/TableFactoryUtil.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/factories/TableFactoryUtil.java index 7adc4762b79b8..440e568278341 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/factories/TableFactoryUtil.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/factories/TableFactoryUtil.java @@ -24,6 +24,8 @@ import org.apache.flink.table.catalog.Catalog; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.descriptors.ConnectorDescriptorValidator; +import org.apache.flink.table.descriptors.DescriptorProperties; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.table.sources.TableSource; @@ -129,4 +131,37 @@ public static Optional createTableSinkForCatalogTable( } return Optional.empty(); } + + /** Checks whether the {@link CatalogTable} uses legacy connector sink options. */ + public static boolean isLegacyConnectorOptions( + @Nullable Catalog catalog, + ReadableConfig configuration, + boolean isStreamingMode, + ObjectIdentifier objectIdentifier, + CatalogTable catalogTable, + boolean isTemporary) { + // normalize option keys + DescriptorProperties properties = new DescriptorProperties(true); + properties.putProperties(catalogTable.getOptions()); + if (properties.containsKey(ConnectorDescriptorValidator.CONNECTOR_TYPE)) { + return true; + } else { + try { + // try to create legacy table source using the options, + // some legacy factories may use the 'type' key + TableFactoryUtil.findAndCreateTableSink( + catalog, + objectIdentifier, + catalogTable, + configuration, + isStreamingMode, + isTemporary); + // success, then we will use the legacy factories + return true; + } catch (Throwable ignore) { + // fail, then we will use new factories + return false; + } + } + } } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/CalculatedTableFactory.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/CalculatedTableFactory.java index 09b9eca32aa44..6d20abddefe3a 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/CalculatedTableFactory.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/CalculatedTableFactory.java @@ -28,6 +28,7 @@ import org.apache.flink.table.expressions.ResolvedExpression; import org.apache.flink.table.expressions.utils.ResolvedExpressionDefaultVisitor; import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.functions.FunctionKind; import org.apache.flink.table.operations.CalculatedQueryOperation; import org.apache.flink.table.operations.QueryOperation; import org.apache.flink.table.types.DataType; @@ -38,6 +39,7 @@ import java.util.List; import static java.util.stream.Collectors.toList; +import static org.apache.flink.table.expressions.ApiExpressionUtils.isFunctionOfKind; import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.AS; /** Utility class for creating a valid {@link CalculatedQueryOperation} operation. */ @@ -89,7 +91,7 @@ private CalculatedQueryOperation unwrapFromAlias(CallExpression call) { + alias))) .collect(toList()); - if (!(children.get(0) instanceof CallExpression)) { + if (!isFunctionOfKind(children.get(0), FunctionKind.TABLE)) { throw fail(); } @@ -156,7 +158,7 @@ protected CalculatedQueryOperation defaultMethod(ResolvedExpression expression) private ValidationException fail() { return new ValidationException( - "A lateral join only accepts a string expression which defines a table function " + "A lateral join only accepts an expression which defines a table function " + "call that might be followed by some alias."); } } diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/EnvironmentSettingsTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/EnvironmentSettingsTest.java index bc2428627fa3b..ad4630b6c08e4 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/EnvironmentSettingsTest.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/EnvironmentSettingsTest.java @@ -34,15 +34,16 @@ public class EnvironmentSettingsTest { public void testFromConfiguration() { Configuration configuration = new Configuration(); configuration.setString("execution.runtime-mode", "batch"); - EnvironmentSettings settings = EnvironmentSettings.fromConfiguration(configuration); + EnvironmentSettings settings = + EnvironmentSettings.newInstance().withConfiguration(configuration).build(); assertFalse("Expect batch mode.", settings.isStreamingMode()); } @Test - public void testToConfiguration() { + public void testGetConfiguration() { EnvironmentSettings settings = new EnvironmentSettings.Builder().inBatchMode().build(); - Configuration configuration = settings.toConfiguration(); + Configuration configuration = settings.getConfiguration(); assertEquals(RuntimeExecutionMode.BATCH, configuration.get(RUNTIME_MODE)); } diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java index c94dee9d798c3..b30f80665e473 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java @@ -18,7 +18,7 @@ package org.apache.flink.table.catalog; -import org.apache.flink.table.api.TableConfig; +import org.apache.flink.configuration.Configuration; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.functions.AggregateFunction; import org.apache.flink.table.functions.FunctionDefinition; @@ -85,7 +85,7 @@ public void init() { functionCatalog = new FunctionCatalog( - TableConfig.getDefault(), + new Configuration(), CatalogManagerMocks.preparedCatalogManager() .defaultCatalog(DEFAULT_CATALOG, catalog) .build(), diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/SchemaResolutionTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/SchemaResolutionTest.java index 7bf66e96b0855..2e50f73ae8108 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/SchemaResolutionTest.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/SchemaResolutionTest.java @@ -221,6 +221,26 @@ public void testSchemaResolutionErrors() { Schema.newBuilder().columnByExpression("invalid", callSql("INVALID")).build(), "Invalid expression for computed column 'invalid'."); + // metadata columns + + testError( + Schema.newBuilder() + .columnByMetadata("metadata", DataTypes.INT()) + .columnByMetadata("from_metadata", DataTypes.BIGINT(), "metadata", false) + .build(), + "The column `metadata` and `from_metadata` in the table are both from the same metadata key 'metadata'. " + + "Please specify one of the columns as the metadata column and use the " + + "computed column syntax to specify the others."); + + testError( + Schema.newBuilder() + .columnByMetadata("from_metadata", DataTypes.BIGINT(), "metadata", false) + .columnByMetadata("from_metadata2", DataTypes.STRING(), "metadata", true) + .build(), + "The column `from_metadata` and `from_metadata2` in the table are both from the same metadata key 'metadata'. " + + "Please specify one of the columns as the metadata column and use the " + + "computed column syntax to specify the others."); + // time attributes and watermarks testError( diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/CatalogManagerMocks.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/CatalogManagerMocks.java index 94bc1823a4c13..c3fb2e45ae456 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/CatalogManagerMocks.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/CatalogManagerMocks.java @@ -20,7 +20,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.catalog.Catalog; import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.catalog.GenericInMemoryCatalog; @@ -30,9 +30,11 @@ /** Mock implementations of {@link CatalogManager} for testing purposes. */ public final class CatalogManagerMocks { - public static final String DEFAULT_CATALOG = EnvironmentSettings.DEFAULT_BUILTIN_CATALOG; + public static final String DEFAULT_CATALOG = + TableConfigOptions.TABLE_CATALOG_NAME.defaultValue(); - public static final String DEFAULT_DATABASE = EnvironmentSettings.DEFAULT_BUILTIN_DATABASE; + public static final String DEFAULT_DATABASE = + TableConfigOptions.TABLE_DATABASE_NAME.defaultValue(); public static CatalogManager createEmptyCatalogManager() { return createCatalogManager(null); diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/TableEnvironmentMock.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/TableEnvironmentMock.java index cb09387a842ff..9403063318861 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/TableEnvironmentMock.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/TableEnvironmentMock.java @@ -18,6 +18,7 @@ package org.apache.flink.table.utils; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.internal.TableEnvironmentImpl; @@ -69,7 +70,7 @@ public static TableEnvironmentMock getBatchInstance() { } private static TableEnvironmentMock getInstance(boolean isStreamingMode) { - final TableConfig tableConfig = createTableConfig(); + final TableConfig tableConfig = TableConfig.getDefault(); final CatalogManager catalogManager = CatalogManagerMocks.createEmptyCatalogManager(); final ModuleManager moduleManager = new ModuleManager(); return new TableEnvironmentMock( @@ -82,17 +83,13 @@ private static TableEnvironmentMock getInstance(boolean isStreamingMode) { isStreamingMode); } - private static TableConfig createTableConfig() { - return TableConfig.getDefault(); - } - private static ExecutorMock createExecutor() { return new ExecutorMock(); } private static FunctionCatalog createFunctionCatalog( - TableConfig tableConfig, CatalogManager catalogManager, ModuleManager moduleManager) { - return new FunctionCatalog(tableConfig, catalogManager, moduleManager); + ReadableConfig config, CatalogManager catalogManager, ModuleManager moduleManager) { + return new FunctionCatalog(config, catalogManager, moduleManager); } private static PlannerMock createPlanner() { diff --git a/flink-table/flink-table-api-scala-bridge/pom.xml b/flink-table/flink-table-api-scala-bridge/pom.xml index 9dd3d220721d5..539acc460dcc4 100644 --- a/flink-table/flink-table-api-scala-bridge/pom.xml +++ b/flink-table/flink-table-api-scala-bridge/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/StreamTableEnvironment.scala b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/StreamTableEnvironment.scala index 49d0b7a556f3a..345e2f921ca11 100644 --- a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/StreamTableEnvironment.scala +++ b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/StreamTableEnvironment.scala @@ -18,7 +18,6 @@ package org.apache.flink.table.api.bridge.scala import org.apache.flink.annotation.PublicEvolving -import org.apache.flink.api.common.JobExecutionResult import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.common.typeutils.CompositeType import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment} @@ -869,9 +868,7 @@ object StreamTableEnvironment { * [[TableEnvironment]]. */ def create(executionEnvironment: StreamExecutionEnvironment): StreamTableEnvironment = { - create( - executionEnvironment, - EnvironmentSettings.fromConfiguration(executionEnvironment.getConfiguration)) + create(executionEnvironment, EnvironmentSettings.newInstance().build) } /** @@ -899,43 +896,6 @@ object StreamTableEnvironment { executionEnvironment: StreamExecutionEnvironment, settings: EnvironmentSettings) : StreamTableEnvironment = { - val config = new TableConfig() - config.addConfiguration(settings.toConfiguration) - StreamTableEnvironmentImpl - .create(executionEnvironment, settings, config) - } - - /** - * Creates a table environment that is the entry point and central context for creating Table and - * SQL API programs that integrate with the Scala-specific [[DataStream]] API. - * - * It is unified for bounded and unbounded data processing. - * - * A stream table environment is responsible for: - * - * - Convert a [[DataStream]] into [[Table]] and vice-versa. - * - Connecting to external systems. - * - Registering and retrieving [[Table]]s and other meta objects from a catalog. - * - Executing SQL statements. - * - Offering further configuration options. - * - * Note: If you don't intend to use the [[DataStream]] API, [[TableEnvironment]] is meant for - * pure table programs. - * - * @param executionEnvironment The Scala [[StreamExecutionEnvironment]] of the - * [[TableEnvironment]]. - * @param tableConfig The configuration of the [[TableEnvironment]]. - * @deprecated Use [[create(StreamExecutionEnvironment)]] and - * [[StreamTableEnvironment#getConfig()]] for manipulating the [[TableConfig]]. - */ - @deprecated - def create(executionEnvironment: StreamExecutionEnvironment, tableConfig: TableConfig) - : StreamTableEnvironment = { - - StreamTableEnvironmentImpl - .create( - executionEnvironment, - EnvironmentSettings.fromConfiguration(tableConfig.getConfiguration), - tableConfig) + StreamTableEnvironmentImpl.create(executionEnvironment, settings) } } diff --git a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/internal/StreamTableEnvironmentImpl.scala b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/internal/StreamTableEnvironmentImpl.scala index f56ba48f10278..1c22f5dd92197 100644 --- a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/internal/StreamTableEnvironmentImpl.scala +++ b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/internal/StreamTableEnvironmentImpl.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.table.api.bridge.scala.internal -import org.apache.flink.annotation.Internal +import org.apache.flink.annotation.{Internal, VisibleForTesting} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala._ import org.apache.flink.streaming.api.TimeCharacteristic @@ -40,6 +40,7 @@ import org.apache.flink.types.Row import org.apache.flink.util.Preconditions import java.util.Optional + import scala.collection.JavaConverters._ /** @@ -294,18 +295,24 @@ object StreamTableEnvironmentImpl { def create( executionEnvironment: StreamExecutionEnvironment, - settings: EnvironmentSettings, - tableConfig: TableConfig) + settings: EnvironmentSettings) : StreamTableEnvironmentImpl = { // temporary solution until FLINK-15635 is fixed val classLoader = Thread.currentThread.getContextClassLoader + val executor = AbstractStreamTableEnvironmentImpl.lookupExecutor( + classLoader, executionEnvironment.getWrappedStreamExecutionEnvironment) + + val tableConfig = TableConfig.getDefault + tableConfig.setRootConfiguration(executor.getConfiguration) + tableConfig.addConfiguration(settings.getConfiguration) + val moduleManager = new ModuleManager val catalogManager = CatalogManager.newBuilder .classLoader(classLoader) - .config(tableConfig.getConfiguration) + .config(tableConfig) .defaultCatalog( settings.getBuiltInCatalogName, new GenericInMemoryCatalog( @@ -316,11 +323,8 @@ object StreamTableEnvironmentImpl { val functionCatalog = new FunctionCatalog(tableConfig, catalogManager, moduleManager) - val executor = AbstractStreamTableEnvironmentImpl.lookupExecutor( - classLoader, settings.getExecutor, executionEnvironment.getWrappedStreamExecutionEnvironment) - - val planner = PlannerFactoryUtil.createPlanner(settings.getPlanner, executor, tableConfig, - moduleManager, catalogManager, functionCatalog) + val planner = PlannerFactoryUtil.createPlanner( + executor, tableConfig, moduleManager, catalogManager, functionCatalog) new StreamTableEnvironmentImpl( catalogManager, diff --git a/flink-table/flink-table-api-scala/pom.xml b/flink-table/flink-table-api-scala/pom.xml index 84b79e716ceb6..281bd9ae5c79d 100644 --- a/flink-table/flink-table-api-scala/pom.xml +++ b/flink-table/flink-table-api-scala/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/ImplicitExpressionConversions.scala b/flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/ImplicitExpressionConversions.scala index b2c0ca2bfd29c..c36c3d10deeaa 100644 --- a/flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/ImplicitExpressionConversions.scala +++ b/flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/ImplicitExpressionConversions.scala @@ -158,17 +158,13 @@ trait ImplicitExpressionConversions { } } - implicit class TableFunctionCall[T: TypeInformation](val t: TableFunction[T]) { + implicit class TableFunctionCall(val t: TableFunction[_]) { /** * Calls a table function for the given parameters. */ def apply(params: Expression*): Expression = { - val resultTypeInfo: TypeInformation[T] = UserDefinedFunctionHelper - .getReturnTypeOfTableFunction(t, implicitly[TypeInformation[T]]) - unresolvedCall( - new TableFunctionDefinition(t.getClass.getName, t, resultTypeInfo), - params.map(ApiExpressionUtils.objectToExpression): _*) + unresolvedCall(t, params.map(ApiExpressionUtils.objectToExpression): _*) } } diff --git a/flink-table/flink-table-code-splitter/pom.xml b/flink-table/flink-table-code-splitter/pom.xml index 2be8a7e7111bb..76daebe6014a0 100644 --- a/flink-table/flink-table-code-splitter/pom.xml +++ b/flink-table/flink-table-code-splitter/pom.xml @@ -24,7 +24,7 @@ under the License. flink-table org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-table-common/pom.xml b/flink-table/flink-table-common/pom.xml index 514efd26168a7..a8a211fd82370 100644 --- a/flink-table/flink-table-common/pom.xml +++ b/flink-table/flink-table-common/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/sink/abilities/SupportsWritingMetadata.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/sink/abilities/SupportsWritingMetadata.java index dd344950c1dc5..d789da1d842f6 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/sink/abilities/SupportsWritingMetadata.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/sink/abilities/SupportsWritingMetadata.java @@ -68,13 +68,17 @@ * casting from INT will be performed by the planner in a preceding operation: * *
        {@code
        - * // for t1 and t2
        - * ROW < i INT, s STRING, d DOUBLE >                                              // physical input
        - * ROW < i INT, s STRING, d DOUBLE, timestamp TIMESTAMP(3) WITH LOCAL TIME ZONE > // final input
        + * // physical input
        + * ROW < i INT, s STRING, d DOUBLE >
          *
        - * // for t3
        - * ROW < i INT, s STRING, d DOUBLE >                                              // physical input
        - * ROW < i INT, s STRING, d DOUBLE >                                              // final input
        + * // final input (i.e. consumed type) for t1
        + * ROW < i INT, s STRING, d DOUBLE, timestamp TIMESTAMP(3) WITH LOCAL TIME ZONE >
        + *
        + * // final input (i.e. consumed type) for t2
        + * ROW < i INT, s STRING, d DOUBLE, myTimestamp TIMESTAMP(3) WITH LOCAL TIME ZONE >
        + *
        + * // final input (i.e. consumed type) for t3
        + * ROW < i INT, s STRING, d DOUBLE >
          * }
        */ @PublicEvolving @@ -115,7 +119,8 @@ public interface SupportsWritingMetadata { * * @param metadataKeys a subset of the keys returned by {@link #listWritableMetadata()}, ordered * by the iteration order of returned map - * @param consumedDataType the final input type of the sink + * @param consumedDataType the final input type of the sink, it is intended to be only forwarded + * and the planner will decide on the field names to avoid collisions * @see EncodingFormat#applyWritableMetadata(List) */ void applyWritableMetadata(List metadataKeys, DataType consumedDataType); diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/abilities/SupportsReadingMetadata.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/abilities/SupportsReadingMetadata.java index dcf20353af4fb..60b6932b61769 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/abilities/SupportsReadingMetadata.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/abilities/SupportsReadingMetadata.java @@ -76,9 +76,14 @@ * casting to INT will be performed by the planner in a subsequent operation: * *
        {@code
        - * // for t1 and t2
        - * ROW < i INT, s STRING, d DOUBLE >                                              // physical output
        - * ROW < i INT, s STRING, d DOUBLE, timestamp TIMESTAMP(3) WITH LOCAL TIME ZONE > // final output
        + * // physical output
        + * ROW < i INT, s STRING, d DOUBLE >
        + *
        + * // final output (i.e. produced type) for t1
        + * ROW < i INT, s STRING, d DOUBLE, timestamp TIMESTAMP(3) WITH LOCAL TIME ZONE >
        + *
        + * // final output (i.e. produced type) for t2
        + * ROW < i INT, s STRING, d DOUBLE, myTimestamp TIMESTAMP(3) WITH LOCAL TIME ZONE >
          * }
        */ @PublicEvolving @@ -129,7 +134,8 @@ public interface SupportsReadingMetadata { * * @param metadataKeys a subset of the keys returned by {@link #listReadableMetadata()}, ordered * by the iteration order of returned map - * @param producedDataType the final output type of the source + * @param producedDataType the final output type of the source, it is intended to be only + * forwarded and the planner will decide on the field names to avoid collisions * @see DecodingFormat#applyReadableMetadata(List) */ void applyReadableMetadata(List metadataKeys, DataType producedDataType); diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/GenericArrayData.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/GenericArrayData.java index 841e0f7fb42f8..6c694799f38fb 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/GenericArrayData.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/GenericArrayData.java @@ -29,13 +29,28 @@ /** * An internal data structure representing data of {@link ArrayType}. * + *

        Note: All elements of this data structure must be internal data structures and must be of the + * same type. See {@link RowData} for more information about internal data structures. + * *

        {@link GenericArrayData} is a generic implementation of {@link ArrayData} which wraps regular * Java arrays. * - *

        Note: All elements of this data structure must be internal data structures and must be of the - * same type. See {@link RowData} for more information about internal data structures. + *

        Every instance wraps a one-dimensional Java array. Non-primitive arrays can be used for + * representing element nullability. The Java array might be a primitive array such as {@code int[]} + * or an object array (i.e. instance of {@code Object[]}). Object arrays that contain boxed types + * (e.g. {@link Integer}) MUST be boxed arrays (i.e. {@code new Integer[]{1, 2, 3}}, not {@code new + * Object[]{1, 2, 3}}). For multidimensional arrays, an array of {@link GenericArrayData} MUST be + * passed. For example: * - *

        For non-primitive arrays, elements can contain null for representing nullability. + *

        {@code
        + * // ARRAY < ARRAY < INT NOT NULL > >
        + * new GenericArrayData(
        + *   new GenericArrayData[]{
        + *     new GenericArrayData(new int[3]),
        + *     new GenericArrayData(new int[5])
        + *   }
        + * )
        + * }
        */ @PublicEvolving public final class GenericArrayData implements ArrayData { diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java index e947376891772..725d823a1bf90 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java @@ -1720,7 +1720,7 @@ ANY, and(logical(LogicalTypeRoot.BOOLEAN), LITERAL) public static final BuiltInFunctionDefinition TRY_CAST = BuiltInFunctionDefinition.newBuilder() - .name("tryCast") + .name("TRY_CAST") .kind(SCALAR) .inputTypeStrategy(SpecificInputTypeStrategies.CAST) .outputTypeStrategy(forceNullable(TypeStrategies.argument(1))) diff --git a/flink-table/flink-table-planner-loader-bundle/pom.xml b/flink-table/flink-table-planner-loader-bundle/pom.xml index 5df0d1284fbc0..4c9f772909666 100644 --- a/flink-table/flink-table-planner-loader-bundle/pom.xml +++ b/flink-table/flink-table-planner-loader-bundle/pom.xml @@ -26,7 +26,7 @@ org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-table-planner-loader/pom.xml b/flink-table/flink-table-planner-loader/pom.xml index 8a1a7746777e7..19bdf7e36d9d8 100644 --- a/flink-table/flink-table-planner-loader/pom.xml +++ b/flink-table/flink-table-planner-loader/pom.xml @@ -26,7 +26,7 @@ org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-table-planner/pom.xml b/flink-table/flink-table-planner/pom.xml index 469a7a3d591b2..9f4cfeef174ac 100644 --- a/flink-table/flink-table-planner/pom.xml +++ b/flink-table/flink-table-planner/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/ProcedureNamespace.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/ProcedureNamespace.java index cf9beec2f45cb..22e93800aeb9a 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/ProcedureNamespace.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/ProcedureNamespace.java @@ -21,13 +21,12 @@ import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.sql.SqlCall; import org.apache.calcite.sql.SqlCallBinding; -import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlOperator; import org.apache.calcite.sql.SqlTableFunction; -import org.apache.calcite.sql.SqlUtil; import org.apache.calcite.sql.type.SqlReturnTypeInference; -import org.apache.calcite.sql.type.SqlTypeName; + +import static java.util.Objects.requireNonNull; /** * Namespace whose contents are defined by the result of a call to a user-defined procedure. @@ -56,25 +55,23 @@ public final class ProcedureNamespace extends AbstractNamespace { public RelDataType validateImpl(RelDataType targetRowType) { validator.inferUnknownTypes(validator.unknownType, scope, call); - final RelDataType type = validator.deriveTypeImpl(scope, call); + // The result is ignored but the type is derived to trigger the validation + validator.deriveTypeImpl(scope, call); final SqlOperator operator = call.getOperator(); final SqlCallBinding callBinding = new SqlCallBinding(validator, scope, call); - if (operator instanceof SqlTableFunction) { - final SqlTableFunction tableFunction = (SqlTableFunction) operator; - if (type.getSqlTypeName() != SqlTypeName.CURSOR) { - throw new IllegalArgumentException( - "Table function should have CURSOR " + "type, not " + type); - } - final SqlReturnTypeInference rowTypeInference = tableFunction.getRowTypeInference(); - RelDataType retType = rowTypeInference.inferReturnType(callBinding); - return validator.getTypeFactory().createTypeWithNullability(retType, false); - } - - // special handling of collection tables TABLE(function(...)) - if (SqlUtil.stripAs(enclosingNode).getKind() == SqlKind.COLLECTION_TABLE) { - return toStruct(type, getNode()); + if (!(operator instanceof SqlTableFunction)) { + throw new IllegalArgumentException( + "Argument must be a table function: " + operator.getNameAsId()); } - return type; + final SqlTableFunction tableFunction = (SqlTableFunction) operator; + final SqlReturnTypeInference rowTypeInference = tableFunction.getRowTypeInference(); + final RelDataType rowRelDataType = + requireNonNull( + rowTypeInference.inferReturnType(callBinding), + () -> "got null from inferReturnType for call " + callBinding.getCall()); + // For BridgingSqlFunction the type can still be atomic + // and will be wrapped with a proper field alias + return toStruct(rowRelDataType, getNode()); } /** Converts a type to a struct if it is not already. */ diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/FlinkRelBuilder.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/FlinkRelBuilder.java new file mode 100644 index 0000000000000..c5b774de45f41 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/FlinkRelBuilder.java @@ -0,0 +1,294 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.calcite; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.operations.QueryOperation; +import org.apache.flink.table.planner.calcite.FlinkRelFactories.ExpandFactory; +import org.apache.flink.table.planner.calcite.FlinkRelFactories.RankFactory; +import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction; +import org.apache.flink.table.planner.hint.FlinkHints; +import org.apache.flink.table.planner.plan.QueryOperationConverter; +import org.apache.flink.table.planner.plan.logical.LogicalWindow; +import org.apache.flink.table.planner.plan.nodes.calcite.LogicalTableAggregate; +import org.apache.flink.table.planner.plan.nodes.calcite.LogicalWatermarkAssigner; +import org.apache.flink.table.planner.plan.nodes.calcite.LogicalWindowAggregate; +import org.apache.flink.table.planner.plan.nodes.calcite.LogicalWindowTableAggregate; +import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty; +import org.apache.flink.table.runtime.operators.rank.RankRange; +import org.apache.flink.table.runtime.operators.rank.RankType; +import org.apache.flink.util.CollectionUtil; +import org.apache.flink.util.Preconditions; + +import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList; + +import org.apache.calcite.plan.Context; +import org.apache.calcite.plan.Contexts; +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptRule; +import org.apache.calcite.plan.RelOptSchema; +import org.apache.calcite.plan.RelOptTable.ToRelContext; +import org.apache.calcite.plan.ViewExpanders; +import org.apache.calcite.rel.RelCollation; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.AggregateCall; +import org.apache.calcite.rel.hint.RelHint; +import org.apache.calcite.rel.logical.LogicalAggregate; +import org.apache.calcite.rel.logical.LogicalTableFunctionScan; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.tools.RelBuilder; +import org.apache.calcite.tools.RelBuilderFactory; +import org.apache.calcite.util.ImmutableBitSet; +import org.apache.calcite.util.Util; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.function.UnaryOperator; +import java.util.stream.Collectors; + +import static org.apache.flink.table.planner.plan.utils.AggregateUtil.isTableAggregate; + +/** Flink-specific {@link RelBuilder}. */ +@Internal +public final class FlinkRelBuilder extends RelBuilder { + + private final QueryOperationConverter toRelNodeConverter; + + private final ExpandFactory expandFactory; + + private final RankFactory rankFactory; + + private FlinkRelBuilder(Context context, RelOptCluster cluster, RelOptSchema relOptSchema) { + super(context, cluster, relOptSchema); + + this.toRelNodeConverter = + new QueryOperationConverter(this, context.unwrap(FlinkContext.class).isBatchMode()); + this.expandFactory = + Util.first( + context.unwrap(ExpandFactory.class), + FlinkRelFactories.DEFAULT_EXPAND_FACTORY()); + this.rankFactory = + Util.first( + context.unwrap(RankFactory.class), + FlinkRelFactories.DEFAULT_RANK_FACTORY()); + } + + public static FlinkRelBuilder of( + Context context, RelOptCluster cluster, RelOptSchema relOptSchema) { + return new FlinkRelBuilder(Preconditions.checkNotNull(context), cluster, relOptSchema); + } + + public static FlinkRelBuilder of(RelOptCluster cluster, RelOptSchema relOptSchema) { + return FlinkRelBuilder.of(cluster.getPlanner().getContext(), cluster, relOptSchema); + } + + public static RelBuilderFactory proto(Context context) { + return (cluster, schema) -> { + final Context clusterContext = cluster.getPlanner().getContext(); + final Context chain = Contexts.chain(context, clusterContext); + return FlinkRelBuilder.of(chain, cluster, schema); + }; + } + + /** + * {@link RelBuilder#functionScan(SqlOperator, int, Iterable)} cannot work smoothly with aliases + * which is why we implement a custom one. The method is static because some {@link RelOptRule}s + * don't use {@link FlinkRelBuilder}. + */ + public static RelBuilder pushFunctionScan( + RelBuilder relBuilder, + SqlOperator operator, + int inputCount, + Iterable operands, + List aliases) { + Preconditions.checkArgument( + operator instanceof BridgingSqlFunction.WithTableFunction, + "Table function expected."); + final RexBuilder rexBuilder = relBuilder.getRexBuilder(); + final RelDataTypeFactory typeFactory = relBuilder.getTypeFactory(); + + final List inputs = new LinkedList<>(); + for (int i = 0; i < inputCount; i++) { + inputs.add(0, relBuilder.build()); + } + + final List operandList = CollectionUtil.iterableToList(operands); + + final RelDataType functionRelDataType = rexBuilder.deriveReturnType(operator, operandList); + final List fieldRelDataTypes; + if (functionRelDataType.isStruct()) { + fieldRelDataTypes = + functionRelDataType.getFieldList().stream() + .map(RelDataTypeField::getType) + .collect(Collectors.toList()); + } else { + fieldRelDataTypes = Collections.singletonList(functionRelDataType); + } + final RelDataType rowRelDataType = typeFactory.createStructType(fieldRelDataTypes, aliases); + + final RexNode call = rexBuilder.makeCall(rowRelDataType, operator, operandList); + final RelNode functionScan = + LogicalTableFunctionScan.create( + relBuilder.getCluster(), + inputs, + call, + null, + rowRelDataType, + Collections.emptySet()); + return relBuilder.push(functionScan); + } + + public RelBuilder expand(List> projects, int expandIdIndex) { + final RelNode input = build(); + final RelNode expand = expandFactory.createExpand(input, projects, expandIdIndex); + return push(expand); + } + + public RelBuilder rank( + ImmutableBitSet partitionKey, + RelCollation orderKey, + RankType rankType, + RankRange rankRange, + RelDataTypeField rankNumberType, + boolean outputRankNumber) { + final RelNode input = build(); + final RelNode rank = + rankFactory.createRank( + input, + partitionKey, + orderKey, + rankType, + rankRange, + rankNumberType, + outputRankNumber); + return push(rank); + } + + /** Build non-window aggregate for either aggregate or table aggregate. */ + @Override + public RelBuilder aggregate( + RelBuilder.GroupKey groupKey, Iterable aggCalls) { + // build a relNode, the build() may also return a project + RelNode relNode = super.aggregate(groupKey, aggCalls).build(); + + if (relNode instanceof LogicalAggregate) { + final LogicalAggregate logicalAggregate = (LogicalAggregate) relNode; + if (isTableAggregate(logicalAggregate.getAggCallList())) { + relNode = LogicalTableAggregate.create(logicalAggregate); + } else if (isCountStarAgg(logicalAggregate)) { + final RelNode newAggInput = + push(logicalAggregate.getInput(0)).project(literal(0)).build(); + relNode = + logicalAggregate.copy( + logicalAggregate.getTraitSet(), ImmutableList.of(newAggInput)); + } + } + + return push(relNode); + } + + /** Build window aggregate for either aggregate or table aggregate. */ + public RelBuilder windowAggregate( + LogicalWindow window, + GroupKey groupKey, + List namedProperties, + Iterable aggCalls) { + // build logical aggregate + + // Because of: + // [CALCITE-3763] RelBuilder.aggregate should prune unused fields from the input, + // if the input is a Project. + // + // the field can not be pruned if it is referenced by other expressions + // of the window aggregation(i.e. the TUMBLE_START/END). + // To solve this, we config the RelBuilder to forbidden this feature. + final LogicalAggregate aggregate = + (LogicalAggregate) + super.transform(t -> t.withPruneInputOfAggregate(false)) + .push(build()) + .aggregate(groupKey, aggCalls) + .build(); + + // build logical window aggregate from it + final RelNode windowAggregate; + if (isTableAggregate(aggregate.getAggCallList())) { + windowAggregate = + LogicalWindowTableAggregate.create(window, namedProperties, aggregate); + } else { + windowAggregate = LogicalWindowAggregate.create(window, namedProperties, aggregate); + } + return push(windowAggregate); + } + + /** Build watermark assigner relational node. */ + public RelBuilder watermark(int rowtimeFieldIndex, RexNode watermarkExpr) { + final RelNode input = build(); + final RelNode relNode = + LogicalWatermarkAssigner.create(cluster, input, rowtimeFieldIndex, watermarkExpr); + return push(relNode); + } + + public RelBuilder queryOperation(QueryOperation queryOperation) { + final RelNode relNode = queryOperation.accept(toRelNodeConverter); + return push(relNode); + } + + public RelBuilder scan(ObjectIdentifier identifier, Map dynamicOptions) { + final List hints = new ArrayList<>(); + hints.add( + RelHint.builder(FlinkHints.HINT_NAME_OPTIONS).hintOptions(dynamicOptions).build()); + final ToRelContext toRelContext = ViewExpanders.simpleContext(cluster, hints); + final RelNode relNode = + relOptSchema.getTableForMember(identifier.toList()).toRel(toRelContext); + return push(relNode); + } + + @Override + public FlinkTypeFactory getTypeFactory() { + return (FlinkTypeFactory) super.getTypeFactory(); + } + + @Override + public RelBuilder transform(UnaryOperator transform) { + // Override in order to return a FlinkRelBuilder. + final Context mergedContext = + Contexts.of(transform.apply(Config.DEFAULT), cluster.getPlanner().getContext()); + return FlinkRelBuilder.of(mergedContext, cluster, relOptSchema); + } + + private static boolean isCountStarAgg(LogicalAggregate agg) { + if (agg.getGroupCount() != 0 || agg.getAggCallList().size() != 1) { + return false; + } + final AggregateCall call = agg.getAggCallList().get(0); + return call.getAggregation().getKind() == SqlKind.COUNT + && call.filterArg == -1 + && call.getArgList().isEmpty(); + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/catalog/QueryOperationCatalogViewTable.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/catalog/QueryOperationCatalogViewTable.java index 9ba8e6e7e2e3f..32353c182de87 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/catalog/QueryOperationCatalogViewTable.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/catalog/QueryOperationCatalogViewTable.java @@ -25,6 +25,9 @@ import org.apache.flink.table.planner.plan.schema.ExpandingPreparingTable; import org.apache.flink.table.planner.plan.stats.FlinkStatistic; +import org.apache.calcite.plan.Context; +import org.apache.calcite.plan.Contexts; +import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelOptSchema; import org.apache.calcite.plan.RelOptTable; import org.apache.calcite.rel.RelNode; @@ -70,8 +73,9 @@ public List getQualifiedName() { @Override public RelNode convertToRel(RelOptTable.ToRelContext context) { - FlinkRelBuilder relBuilder = - FlinkRelBuilder.of(context, context.getCluster(), this.getRelOptSchema()); + final RelOptCluster cluster = context.getCluster(); + final Context chain = Contexts.of(context, cluster.getPlanner().getContext()); + final FlinkRelBuilder relBuilder = FlinkRelBuilder.of(chain, cluster, getRelOptSchema()); return relBuilder.queryOperation(catalogView.getQueryOperation()).build(); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSinkUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSinkUtils.java index dd5a362b417db..b810e71c61551 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSinkUtils.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSinkUtils.java @@ -66,9 +66,9 @@ import java.time.ZoneId; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -327,7 +327,8 @@ private static void pushMetadataProjection( Function.identity())); final List metadataColumns = - createRequiredMetadataKeys(schema, sink).stream() + createRequiredMetadataColumns(schema, sink).stream() + .map(col -> col.getMetadataKey().orElse(col.getName())) .map(keyToMetadataColumn::get) .collect(Collectors.toList()); @@ -402,28 +403,31 @@ private static void prepareDynamicSink( } /** - * Returns a list of required metadata keys. Ordered by the iteration order of {@link + * Returns a list of required metadata columns. Ordered by the iteration order of {@link * SupportsWritingMetadata#listWritableMetadata()}. * *

        This method assumes that sink and schema have been validated via {@link * #prepareDynamicSink}. */ - private static List createRequiredMetadataKeys( + private static List createRequiredMetadataColumns( ResolvedSchema schema, DynamicTableSink sink) { final List tableColumns = schema.getColumns(); final List metadataColumns = extractPersistedMetadataColumns(schema); - final Set requiredMetadataKeys = - metadataColumns.stream() - .map(tableColumns::get) - .map(MetadataColumn.class::cast) - .map(c -> c.getMetadataKey().orElse(c.getName())) - .collect(Collectors.toSet()); + Map metadataKeysToMetadataColumns = new HashMap<>(); + + for (Integer columnIndex : metadataColumns) { + MetadataColumn metadataColumn = (MetadataColumn) tableColumns.get(columnIndex); + String metadataKey = metadataColumn.getMetadataKey().orElse(metadataColumn.getName()); + // After resolving, every metadata column has the unique metadata key. + metadataKeysToMetadataColumns.put(metadataKey, metadataColumn); + } final Map metadataMap = extractMetadataMap(sink); return metadataMap.keySet().stream() - .filter(requiredMetadataKeys::contains) + .filter(metadataKeysToMetadataColumns::containsKey) + .map(metadataKeysToMetadataColumns::get) .collect(Collectors.toList()); } @@ -623,7 +627,9 @@ private static void validateAndApplyMetadata( sinkAbilitySpecs.add( new WritingMetadataSpec( - createRequiredMetadataKeys(schema, sink), + createRequiredMetadataColumns(schema, sink).stream() + .map(col -> col.getMetadataKey().orElse(col.getName())) + .collect(Collectors.toList()), createConsumedType(schema, sink))); } @@ -641,8 +647,18 @@ private static RowType createConsumedType(ResolvedSchema schema, DynamicTableSin .map(c -> new RowField(c.getName(), c.getDataType().getLogicalType())); final Stream metadataFields = - createRequiredMetadataKeys(schema, sink).stream() - .map(k -> new RowField(k, metadataMap.get(k).getLogicalType())); + createRequiredMetadataColumns(schema, sink).stream() + .map( + column -> + new RowField( + // Use alias to ensures that physical and metadata + // columns don't collide. + column.getName(), + metadataMap + .get( + column.getMetadataKey() + .orElse(column.getName())) + .getLogicalType())); final List rowFields = Stream.concat(physicalFields, metadataFields).collect(Collectors.toList()); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSourceUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSourceUtils.java index b5fcf2d6e6832..27f93fa6ac412 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSourceUtils.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSourceUtils.java @@ -59,9 +59,9 @@ import org.apache.calcite.rex.RexNode; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -165,26 +165,31 @@ public static void prepareDynamicSource( // TODO: isUpsertSource(), isSourceChangeEventsDuplicate() /** - * Returns a list of required metadata keys. Ordered by the iteration order of {@link + * Returns a list of required metadata columns. Ordered by the iteration order of {@link * SupportsReadingMetadata#listReadableMetadata()}. * *

        This method assumes that source and schema have been validated via {@link * #prepareDynamicSource(String, ResolvedCatalogTable, DynamicTableSource, boolean, * ReadableConfig)}. */ - public static List createRequiredMetadataKeys( + public static List createRequiredMetadataColumns( ResolvedSchema schema, DynamicTableSource source) { final List metadataColumns = extractMetadataColumns(schema); - final Set requiredMetadataKeys = - metadataColumns.stream() - .map(c -> c.getMetadataKey().orElse(c.getName())) - .collect(Collectors.toSet()); + Map metadataKeysToMetadataColumns = new HashMap<>(); + + for (MetadataColumn column : metadataColumns) { + String metadataKey = column.getMetadataKey().orElse(column.getName()); + // After resolving, every metadata column has the unique metadata key. + metadataKeysToMetadataColumns.put(metadataKey, column); + } final Map metadataMap = extractMetadataMap(source); + // reorder the column return metadataMap.keySet().stream() - .filter(requiredMetadataKeys::contains) + .filter(metadataKeysToMetadataColumns::containsKey) + .map(metadataKeysToMetadataColumns::get) .collect(Collectors.toList()); } @@ -203,8 +208,16 @@ public static RowType createProducedType(ResolvedSchema schema, DynamicTableSour ((RowType) schema.toPhysicalRowDataType().getLogicalType()).getFields().stream(); final Stream metadataFields = - createRequiredMetadataKeys(schema, source).stream() - .map(k -> new RowField(k, metadataMap.get(k).getLogicalType())); + createRequiredMetadataColumns(schema, source).stream() + .map( + k -> + new RowField( + // Use the alias to ensure that physical and + // metadata columns don't collide + k.getName(), + metadataMap + .get(k.getMetadataKey().orElse(k.getName())) + .getLogicalType())); final List rowFields = Stream.concat(physicalFields, metadataFields).collect(Collectors.toList()); @@ -237,9 +250,7 @@ public static boolean isSourceChangeEventsDuplicate( boolean isCDCSource = !mode.containsOnly(RowKind.INSERT) && !isUpsertSource(resolvedSchema, tableSource); boolean changeEventsDuplicate = - tableConfig - .getConfiguration() - .getBoolean(ExecutionConfigOptions.TABLE_EXEC_SOURCE_CDC_EVENTS_DUPLICATE); + tableConfig.get(ExecutionConfigOptions.TABLE_EXEC_SOURCE_CDC_EVENTS_DUPLICATE); boolean hasPrimaryKey = resolvedSchema.getPrimaryKey().isPresent(); return isCDCSource && changeEventsDuplicate && hasPrimaryKey; } @@ -312,12 +323,9 @@ private static void pushMetadataProjection(FlinkRelBuilder relBuilder, ResolvedS c.getDataType().getLogicalType()); if (c instanceof MetadataColumn) { final MetadataColumn metadataColumn = (MetadataColumn) c; - final String metadataKey = - metadataColumn - .getMetadataKey() - .orElse(metadataColumn.getName()); + String columnName = metadataColumn.getName(); return rexBuilder.makeAbstractCast( - relDataType, relBuilder.field(metadataKey)); + relDataType, relBuilder.field(columnName)); } else { return relBuilder.field(c.getName()); } @@ -437,7 +445,9 @@ private static void validateAndApplyMetadata( }); metadataSource.applyReadableMetadata( - createRequiredMetadataKeys(schema, source), + createRequiredMetadataColumns(schema, source).stream() + .map(column -> column.getMetadataKey().orElse(column.getName())) + .collect(Collectors.toList()), TypeConversions.fromLogicalToDataType(createProducedType(schema, source))); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/delegation/DefaultPlannerFactory.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/delegation/DefaultPlannerFactory.java index 4476a7a7517f9..b8659ba3250f8 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/delegation/DefaultPlannerFactory.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/delegation/DefaultPlannerFactory.java @@ -29,6 +29,8 @@ import java.util.Collections; import java.util.Set; +import static org.apache.flink.configuration.ExecutionOptions.RUNTIME_MODE; + /** Factory for the default {@link Planner}. */ @Internal public final class DefaultPlannerFactory implements PlannerFactory { @@ -51,7 +53,7 @@ public Set> optionalOptions() { @Override public Planner create(Context context) { final RuntimeExecutionMode runtimeExecutionMode = - context.getTableConfig().getConfiguration().get(ExecutionOptions.RUNTIME_MODE); + context.getTableConfig().get(ExecutionOptions.RUNTIME_MODE); switch (runtimeExecutionMode) { case STREAMING: return new StreamPlanner( @@ -70,8 +72,9 @@ public Planner create(Context context) { default: throw new TableException( String.format( - "Unknown runtime mode '%s'. This is a bug. Please consider filing an issue.", - runtimeExecutionMode)); + "Unsupported mode '%s' for '%s'. Only an explicit BATCH or " + + "STREAMING mode is supported in Table API.", + runtimeExecutionMode, RUNTIME_MODE.key())); } } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/delegation/PlannerConfig.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/delegation/PlannerConfig.java deleted file mode 100644 index 3563d769f65d0..0000000000000 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/delegation/PlannerConfig.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.planner.delegation; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ReadableConfig; - -import java.util.Optional; - -/** - * Configuration view that combines the API specific table configuration and the executor - * configuration. The table configuration has precedence. - */ -@Internal -public final class PlannerConfig implements ReadableConfig { - - private final ReadableConfig tableConfig; - - private final ReadableConfig executorConfig; - - PlannerConfig(ReadableConfig tableConfig, ReadableConfig executorConfig) { - this.tableConfig = tableConfig; - this.executorConfig = executorConfig; - } - - @Override - public T get(ConfigOption option) { - return tableConfig.getOptional(option).orElseGet(() -> executorConfig.get(option)); - } - - @Override - public Optional getOptional(ConfigOption option) { - final Optional tableValue = tableConfig.getOptional(option); - if (tableValue.isPresent()) { - return tableValue; - } - return executorConfig.getOptional(option); - } -} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java index 19a4b6ec34f40..e98e582baa171 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java @@ -184,7 +184,7 @@ public FlinkRelBuilder createRelBuilder(String currentCatalog, String currentDat context, // Sets up the ViewExpander explicitly for FlinkRelBuilder. createFlinkPlanner(currentCatalog, currentDatabase).createToRelContext()); - return new FlinkRelBuilder(chain, cluster, relOptSchema); + return FlinkRelBuilder.of(chain, cluster, relOptSchema); } /** diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/bridging/BridgingSqlFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/bridging/BridgingSqlFunction.java index 5dbe4d8e7ce37..d66178760d8ce 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/bridging/BridgingSqlFunction.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/bridging/BridgingSqlFunction.java @@ -26,6 +26,7 @@ import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.table.functions.FunctionKind; import org.apache.flink.table.planner.calcite.FlinkContext; +import org.apache.flink.table.planner.calcite.FlinkRelBuilder; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; import org.apache.flink.table.planner.utils.ShortcutUtils; import org.apache.flink.table.types.DataType; @@ -35,6 +36,10 @@ import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.sql.SqlFunction; import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.SqlTableFunction; +import org.apache.calcite.sql.type.SqlReturnTypeInference; +import org.apache.calcite.tools.RelBuilder; import java.util.List; @@ -52,7 +57,7 @@ * (either a system or user-defined function). */ @Internal -public final class BridgingSqlFunction extends SqlFunction { +public class BridgingSqlFunction extends SqlFunction { private final DataTypeFactory dataTypeFactory; @@ -108,6 +113,10 @@ public static BridgingSqlFunction of( functionKind == FunctionKind.SCALAR || functionKind == FunctionKind.TABLE, "Scalar or table function kind expected."); + if (functionKind == FunctionKind.TABLE) { + return new BridgingSqlFunction.WithTableFunction( + dataTypeFactory, typeFactory, kind, resolvedFunction, typeInference); + } return new BridgingSqlFunction( dataTypeFactory, typeFactory, kind, resolvedFunction, typeInference); } @@ -177,4 +186,32 @@ public List getParamNames() { public boolean isDeterministic() { return resolvedFunction.getDefinition().isDeterministic(); } + + // -------------------------------------------------------------------------------------------- + // Table function extension + // -------------------------------------------------------------------------------------------- + + /** Special flavor of {@link BridgingSqlFunction} to indicate a table function to Calcite. */ + public static class WithTableFunction extends BridgingSqlFunction implements SqlTableFunction { + + private WithTableFunction( + DataTypeFactory dataTypeFactory, + FlinkTypeFactory typeFactory, + SqlKind kind, + ContextResolvedFunction resolvedFunction, + TypeInference typeInference) { + super(dataTypeFactory, typeFactory, kind, resolvedFunction, typeInference); + } + + /** + * The conversion to a row type is handled on the caller side. This allows us to perform it + * SQL/Table API-specific. This is in particular important to set the aliases of fields + * correctly (see {@link FlinkRelBuilder#pushFunctionScan(RelBuilder, SqlOperator, int, + * Iterable, List)}). + */ + @Override + public SqlReturnTypeInference getRowTypeInference() { + return getReturnTypeInference(); + } + } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/AbstractCodeGeneratorCastRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/AbstractCodeGeneratorCastRule.java index 7ef1a2852a361..90c5f226dd8e9 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/AbstractCodeGeneratorCastRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/AbstractCodeGeneratorCastRule.java @@ -186,6 +186,11 @@ private CastExecutorCodeGeneratorContext(CastRule.Context castRuleCtx) { this.castRuleCtx = castRuleCtx; } + @Override + public boolean isPrinting() { + return castRuleCtx.isPrinting(); + } + @Override public boolean legacyBehaviour() { return castRuleCtx.legacyBehaviour(); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/AbstractExpressionCodeGeneratorCastRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/AbstractExpressionCodeGeneratorCastRule.java index c28d6463803ee..4b05d0bbf0832 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/AbstractExpressionCodeGeneratorCastRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/AbstractExpressionCodeGeneratorCastRule.java @@ -103,6 +103,11 @@ public CastExecutor create( private static CodeGeneratorCastRule.Context createCodeGeneratorCastRuleContext( CastRule.Context ctx) { return new CodeGeneratorCastRule.Context() { + @Override + public boolean isPrinting() { + return ctx.isPrinting(); + } + @Override public boolean legacyBehaviour() { return ctx.legacyBehaviour(); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/ArrayToArrayCastRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/ArrayToArrayCastRule.java index b67b76a566311..f3478af123108 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/ArrayToArrayCastRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/ArrayToArrayCastRule.java @@ -20,8 +20,8 @@ import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.planner.codegen.CodeGenUtils; import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.DistinctType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.LogicalTypeRoot; @@ -109,30 +109,8 @@ protected String generateCodeBlockInternal( private static String arrayElementType(LogicalType t) { if (t.isNullable()) { - return "Object"; + return CodeGenUtils.boxedTypeTermForType(t); } - switch (t.getTypeRoot()) { - case BOOLEAN: - return "boolean"; - case TINYINT: - return "byte"; - case SMALLINT: - return "short"; - case INTEGER: - case DATE: - case TIME_WITHOUT_TIME_ZONE: - case INTERVAL_YEAR_MONTH: - return "int"; - case BIGINT: - case INTERVAL_DAY_TIME: - return "long"; - case FLOAT: - return "float"; - case DOUBLE: - return "double"; - case DISTINCT_TYPE: - return arrayElementType(((DistinctType) t).getSourceType()); - } - return "Object"; + return CodeGenUtils.primitiveTypeTermForType(t); } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/BinaryToStringCastRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/BinaryToStringCastRule.java index b4182f0bf431a..ccb3073ce40f3 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/BinaryToStringCastRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/BinaryToStringCastRule.java @@ -50,25 +50,26 @@ private BinaryToStringCastRule() { isNull$0 = _myInputIsNull; if (!isNull$0) { - java.lang.String hexString$0; - hexString$0 = org.apache.flink.table.utils.EncodingUtils.hex(_myInput); - java.lang.String resultString$152; - resultString$152 = hexString$0.toString(); - if (hexString$0.length() > 3) { - resultString$152 = hexString$0.substring(0, java.lang.Math.min(hexString$0.length(), 3)); + java.lang.String resultString$435; + resultString$435 = new java.lang.String(_myInput, java.nio.charset.StandardCharsets.UTF_8); + java.lang.String resultPadOrTrim$538; + resultPadOrTrim$538 = resultString$435.toString(); + if (resultString$435.length() > 12) { + resultPadOrTrim$538 = resultString$435.substring(0, java.lang.Math.min(resultString$435.length(), 12)); } else { - if (resultString$1.length() < 12) { - int padLength$3; - padLength$3 = 12 - resultString$152.length(); - java.lang.StringBuilder sbPadding$4; - sbPadding$4 = new java.lang.StringBuilder(); - for (int i$5 = 0; i$5 < padLength$3; i$5++) { - sbPadding$4.append(" "); + if (resultPadOrTrim$538.length() < 12) { + int padLength$539; + padLength$539 = 12 - resultPadOrTrim$538.length(); + java.lang.StringBuilder sbPadding$540; + sbPadding$540 = new java.lang.StringBuilder(); + for (int i$541 = 0; i$541 < padLength$539; i$541++) { + sbPadding$540.append(" "); } - resultString$152 = resultString$152 + sbPadding$4.toString(); + resultPadOrTrim$538 = resultPadOrTrim$538 + sbPadding$540.toString(); } } - result$1 = org.apache.flink.table.data.binary.BinaryStringData.fromString(resultString$152); + resultString$435 = resultPadOrTrim$538; + result$1 = org.apache.flink.table.data.binary.BinaryStringData.fromString(resultString$435); isNull$0 = result$1 == null; } else { result$1 = org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8; @@ -84,28 +85,34 @@ protected String generateCodeBlockInternal( LogicalType inputLogicalType, LogicalType targetLogicalType) { final String resultStringTerm = newName("resultString"); - CastRuleUtils.CodeWriter writer = new CastRuleUtils.CodeWriter(); - if (context.legacyBehaviour()) { - writer.declStmt(String.class, resultStringTerm) - .assignStmt( - resultStringTerm, - constructorCall( - String.class, - inputTerm, - accessStaticField(StandardCharsets.class, "UTF_8"))); + final CastRuleUtils.CodeWriter writer = new CastRuleUtils.CodeWriter(); + + writer.declStmt(String.class, resultStringTerm); + if (context.isPrinting()) { + writer.assignStmt(resultStringTerm, "\"x'\"") + .assignPlusStmt( + resultStringTerm, staticCall(EncodingUtils.class, "hex", inputTerm)) + .assignPlusStmt(resultStringTerm, "\"'\""); } else { - final int length = LogicalTypeChecks.getLength(targetLogicalType); + writer.assignStmt( + resultStringTerm, + constructorCall( + String.class, + inputTerm, + accessStaticField(StandardCharsets.class, "UTF_8"))); + } - final String hexStringTerm = newName("hexString"); - writer.declStmt(String.class, hexStringTerm) - .assignStmt(hexStringTerm, staticCall(EncodingUtils.class, "hex", inputTerm)); + if (!context.legacyBehaviour() && !context.isPrinting()) { + final String resultPadOrTrim = newName("resultPadOrTrim"); + final int length = LogicalTypeChecks.getLength(targetLogicalType); CharVarCharTrimPadCastRule.padAndTrimStringIfNeeded( writer, targetLogicalType, context.legacyBehaviour(), length, - resultStringTerm, - hexStringTerm); + resultPadOrTrim, + resultStringTerm); + writer.assignStmt(resultStringTerm, resultPadOrTrim); } return writer // Assign the result value diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRule.java index 57cf29ed2bc67..6b6e036d763fb 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRule.java @@ -34,7 +34,7 @@ @Internal public interface CastRule { - /** @see CastRulePredicate for more details about a cast rule predicate definition */ + /** @see CastRulePredicate for more details about a cast rule predicate definition. */ CastRulePredicate getPredicateDefinition(); /** @@ -50,6 +50,9 @@ CastExecutor create( /** Casting context. */ interface Context { + + boolean isPrinting(); + @Deprecated boolean legacyBehaviour(); @@ -58,8 +61,17 @@ interface Context { ClassLoader getClassLoader(); /** Create a casting context. */ - static Context create(boolean legacyBehaviour, ZoneId zoneId, ClassLoader classLoader) { + static Context create( + boolean isPrinting, + boolean legacyBehaviour, + ZoneId zoneId, + ClassLoader classLoader) { return new Context() { + @Override + public boolean isPrinting() { + return isPrinting; + } + @Override public boolean legacyBehaviour() { return legacyBehaviour; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRuleProvider.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRuleProvider.java index 9ef2abd127b69..c2681f8dab6cb 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRuleProvider.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRuleProvider.java @@ -165,6 +165,29 @@ public static boolean canFail(LogicalType inputType, LogicalType targetType) { context, inputTerm, inputIsNullTerm, inputLogicalType, targetLogicalType); } + /** + * Create a {@link CastExecutor} and execute the cast on the provided {@code value}. Fails with + * {@link IllegalArgumentException} if the rule cannot be resolved, or with an exception from + * the {@link CastExecutor} itself if the rule can fail. + */ + @SuppressWarnings("unchecked") + public static @Nullable Object cast( + CastRule.Context context, + LogicalType inputLogicalType, + LogicalType targetLogicalType, + Object value) { + CastExecutor castExecutor = + (CastExecutor) + CastRuleProvider.create(context, inputLogicalType, targetLogicalType); + + if (castExecutor == null) { + throw new NullPointerException( + "Unsupported casting from " + inputLogicalType + " to " + targetLogicalType); + } + + return castExecutor.cast(value); + } + /** * This method wraps {@link #generateCodeBlock(CodeGeneratorCastRule.Context, String, String, * LogicalType, LogicalType)}, but adding the assumption that the inputTerm is always non-null. diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRuleUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRuleUtils.java index bb1a8dae0f6ef..8468a587f8dea 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRuleUtils.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRuleUtils.java @@ -212,6 +212,10 @@ public CodeWriter assignStmt(String varName, String value) { return stmt(varName + " = " + value); } + public CodeWriter assignPlusStmt(String varName, String value) { + return stmt(varName + " += " + value); + } + public CodeWriter assignArrayStmt(String varName, String index, String value) { return stmt(varName + "[" + index + "] = " + value); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CodeGeneratorCastRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CodeGeneratorCastRule.java index 4dba7141a9280..3f6be48b6b3d0 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CodeGeneratorCastRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CodeGeneratorCastRule.java @@ -44,6 +44,12 @@ CastCodeBlock generateCodeBlock( /** Context for code generation. */ interface Context { + /** + * @return whether it's in printing mode or not. Printing is used by {@link + * RowDataToStringConverterImpl} when printing table row results. + */ + boolean isPrinting(); + /** @return where the legacy behaviour should be followed or not. */ @Deprecated boolean legacyBehaviour(); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/RowDataToStringConverterImpl.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/RowDataToStringConverterImpl.java index 3698e7f553a12..f9a2c2f3e3379 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/RowDataToStringConverterImpl.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/RowDataToStringConverterImpl.java @@ -25,22 +25,23 @@ import org.apache.flink.table.data.utils.CastExecutor; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.VarCharType; import org.apache.flink.table.utils.DateTimeUtils; import org.apache.flink.table.utils.print.PrintStyle; import org.apache.flink.table.utils.print.RowDataToStringConverter; import java.time.ZoneId; import java.util.List; -import java.util.Objects; import java.util.function.Function; -import static org.apache.flink.table.api.DataTypes.STRING; - /** {@link RowData} to {@link String} converter using {@link CastRule}. */ @Internal public final class RowDataToStringConverterImpl implements RowDataToStringConverter { - private final Function[] columnConverters; + private final DataType dataType; + private final CastRule.Context castRuleContext; + + private Function[] columnConverters; @VisibleForTesting public RowDataToStringConverterImpl(DataType dataType) { @@ -51,9 +52,14 @@ public RowDataToStringConverterImpl(DataType dataType) { false); } - @SuppressWarnings("unchecked") public RowDataToStringConverterImpl( DataType dataType, ZoneId zoneId, ClassLoader classLoader, boolean legacyBehaviour) { + this.dataType = dataType; + this.castRuleContext = CastRule.Context.create(true, legacyBehaviour, zoneId, classLoader); + } + + @SuppressWarnings("unchecked") + private void init() { List rowDataTypes = DataType.getFieldDataTypes(dataType); this.columnConverters = new Function[rowDataTypes.size()]; @@ -64,34 +70,29 @@ public RowDataToStringConverterImpl( CastExecutor castExecutor = (CastExecutor) CastRuleProvider.create( - CastRule.Context.create(legacyBehaviour, zoneId, classLoader), - fieldType, - STRING().getLogicalType()); + castRuleContext, fieldType, VarCharType.STRING_TYPE); if (castExecutor == null) { - // Fallback in case no casting rule is defined, for example for MULTISET and - // STRUCTURED - // Links to https://issues.apache.org/jira/browse/FLINK-24403 - this.columnConverters[index] = - row -> { - if (row.isNullAt(index)) { - return PrintStyle.NULL_VALUE; - } - return Objects.toString(getter.getFieldOrNull(row)); - }; - } else { - this.columnConverters[index] = - row -> { - if (row.isNullAt(index)) { - return PrintStyle.NULL_VALUE; - } - return castExecutor.cast(getter.getFieldOrNull(row)).toString(); - }; + throw new IllegalStateException( + "Cannot create a cast executor for converting " + + fieldType + + " to string. This is a bug, please open an issue."); } + this.columnConverters[index] = + row -> { + if (row.isNullAt(index)) { + return PrintStyle.NULL_VALUE; + } + return castExecutor.cast(getter.getFieldOrNull(row)).toString(); + }; } } @Override public String[] convert(RowData rowData) { + if (this.columnConverters == null) { + init(); + } + String[] result = new String[rowData.getArity()]; for (int i = 0; i < result.length; i++) { result[i] = this.columnConverters[i].apply(rowData); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/StringToBinaryCastRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/StringToBinaryCastRule.java index ad95a4192be2d..0bfc2982a24be 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/StringToBinaryCastRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/StringToBinaryCastRule.java @@ -22,14 +22,12 @@ import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.LogicalTypeFamily; import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; -import org.apache.flink.table.utils.EncodingUtils; import static org.apache.flink.table.codesplit.CodeSplitUtil.newName; import static org.apache.flink.table.planner.functions.casting.BinaryToBinaryCastRule.couldPad; import static org.apache.flink.table.planner.functions.casting.BinaryToBinaryCastRule.trimOrPadByteArray; import static org.apache.flink.table.planner.functions.casting.CastRuleUtils.arrayLength; import static org.apache.flink.table.planner.functions.casting.CastRuleUtils.methodCall; -import static org.apache.flink.table.planner.functions.casting.CastRuleUtils.staticCall; /** * {@link LogicalTypeFamily#CHARACTER_STRING} to {@link LogicalTypeFamily#BINARY_STRING} cast rule. @@ -60,8 +58,7 @@ private StringToBinaryCastRule() { // new behavior isNull$0 = _myInputIsNull; if (!isNull$0) { - java.lang.String hexStringTerm$10 = _myInput.toString(); - byte[] byteArrayTerm$0 = org.apache.flink.table.utils.EncodingUtils.decodeHex(hexStringTerm$10); + byte[] byteArrayTerm$0 = _myInput.toBytes(); if (byteArrayTerm$0.length <= 2) { // If could pad result$1 = java.util.Arrays.copyOf(byteArrayTerm$0, 2); @@ -90,14 +87,9 @@ protected String generateCodeBlockInternal( } else { final int targetLength = LogicalTypeChecks.getLength(targetLogicalType); final String byteArrayTerm = newName("byteArrayTerm"); - final String hexStringTerm = newName("hexStringTerm"); return new CastRuleUtils.CodeWriter() - .declStmt(String.class, hexStringTerm, methodCall(inputTerm, "toString")) - .declStmt( - byte[].class, - byteArrayTerm, - staticCall(EncodingUtils.class, "decodeHex", hexStringTerm)) + .declStmt(byte[].class, byteArrayTerm, methodCall(inputTerm, "toBytes")) .ifStmt( arrayLength(byteArrayTerm) + " <= " + targetLength, thenWriter -> { diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java index cd9499610582b..2b18ffee5e3a3 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java @@ -1140,7 +1140,7 @@ public List getAuxiliaryFunctions() { // JSON FUNCTIONS public static final SqlFunction JSON_EXISTS = SqlStdOperatorTable.JSON_EXISTS; - public static final SqlFunction JSON_VALUE = SqlStdOperatorTable.JSON_VALUE; + public static final SqlFunction JSON_VALUE = new SqlJsonValueFunctionWrapper("JSON_VALUE"); public static final SqlFunction JSON_QUERY = new SqlJsonQueryFunctionWrapper(); public static final SqlFunction JSON_OBJECT = new SqlJsonObjectFunctionWrapper(); public static final SqlAggFunction JSON_OBJECTAGG_NULL_ON_NULL = diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/SqlJsonValueFunctionWrapper.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/SqlJsonValueFunctionWrapper.java new file mode 100644 index 0000000000000..b28ef4786e47f --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/SqlJsonValueFunctionWrapper.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.functions.sql; + +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.sql.SqlJsonValueReturning; +import org.apache.calcite.sql.SqlOperatorBinding; +import org.apache.calcite.sql.fun.SqlJsonValueFunction; +import org.apache.calcite.sql.type.ReturnTypes; +import org.apache.calcite.sql.type.SqlReturnTypeInference; +import org.apache.calcite.sql.type.SqlTypeTransforms; + +import java.util.Optional; + +import static org.apache.flink.table.planner.plan.type.FlinkReturnTypes.VARCHAR_FORCE_NULLABLE; + +/** + * This class is a wrapper class for the {@link SqlJsonValueFunction} but using the {@code + * VARCHAR_FORCE_NULLABLE} return type inference by default. It also supports specifying return type + * with the RETURNING keyword just like the original {@link SqlJsonValueFunction}. + */ +class SqlJsonValueFunctionWrapper extends SqlJsonValueFunction { + + private final SqlReturnTypeInference returnTypeInference; + + SqlJsonValueFunctionWrapper(String name) { + super(name); + this.returnTypeInference = + ReturnTypes.cascade( + SqlJsonValueFunctionWrapper::explicitTypeSpec, + SqlTypeTransforms.FORCE_NULLABLE) + .orElse(VARCHAR_FORCE_NULLABLE); + } + + @Override + public RelDataType inferReturnType(SqlOperatorBinding opBinding) { + RelDataType returnType = returnTypeInference.inferReturnType(opBinding); + if (returnType == null) { + throw new IllegalArgumentException( + "Cannot infer return type for " + + opBinding.getOperator() + + "; operand types: " + + opBinding.collectOperandTypes()); + } + return returnType; + } + + @Override + public SqlReturnTypeInference getReturnTypeInference() { + return returnTypeInference; + } + + /** + * Copied and modified from the original {@link SqlJsonValueFunction}. + * + *

        Changes: Instead of returning {@link Optional} this method returns null directly. + */ + private static RelDataType explicitTypeSpec(SqlOperatorBinding opBinding) { + if (opBinding.getOperandCount() > 2 + && opBinding.isOperandLiteral(2, false) + && opBinding.getOperandLiteralValue(2, Object.class) + instanceof SqlJsonValueReturning) { + return opBinding.getOperandType(3); + } + return null; + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/SqlTryCastFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/SqlTryCastFunction.java index fffb5fe5b0618..5c307e606f8c1 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/SqlTryCastFunction.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/SqlTryCastFunction.java @@ -33,8 +33,6 @@ import org.apache.calcite.sql.fun.SqlCastFunction; import org.apache.calcite.sql.fun.SqlStdOperatorTable; -import static org.apache.flink.table.functions.BuiltInFunctionDefinition.DEFAULT_VERSION; - /** * This class implements the {@code TRY_CAST} built-in, essentially delegating all the method * invocations, whenever is possible, to Calcite's {@link SqlCastFunction}. @@ -49,16 +47,12 @@ public class SqlTryCastFunction extends BuiltInSqlFunction { SqlTryCastFunction() { super( "TRY_CAST", - DEFAULT_VERSION, SqlKind.OTHER_FUNCTION, null, SqlStdOperatorTable.CAST .getOperandTypeInference(), // From Calcite's SqlCastFunction null, - SqlFunctionCategory.SYSTEM, - true, - false, - SqlStdOperatorTable.CAST::getMonotonicity); + SqlFunctionCategory.SYSTEM); } @Override diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java index e688f23437611..3fcf62fafbe3e 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java @@ -19,7 +19,6 @@ package org.apache.flink.table.planner.plan; import org.apache.flink.annotation.Internal; -import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.TableSchema; @@ -306,10 +305,14 @@ public RelNode visit(CalculatedQueryOperation calculatedTable) { final BridgingSqlFunction sqlFunction = BridgingSqlFunction.of(relBuilder.getCluster(), resolvedFunction); - return relBuilder - .functionScan(sqlFunction, 0, parameters) - .rename(calculatedTable.getResolvedSchema().getColumnNames()) - .build(); + FlinkRelBuilder.pushFunctionScan( + relBuilder, + sqlFunction, + 0, + parameters, + calculatedTable.getResolvedSchema().getColumnNames()); + + return relBuilder.build(); } private RelNode convertLegacyTableFunction( @@ -527,10 +530,9 @@ private RelNode convertToExternalScan( boolean isTopLevelRecord, ChangelogMode changelogMode) { final FlinkContext flinkContext = ShortcutUtils.unwrapContext(relBuilder); - final ReadableConfig config = flinkContext.getTableConfig().getConfiguration(); return DynamicSourceUtils.convertDataStreamToRel( flinkContext.isBatchMode(), - config, + flinkContext.getTableConfig(), relBuilder, contextResolvedTable, dataStream, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/abilities/source/WatermarkPushDownSpec.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/abilities/source/WatermarkPushDownSpec.java index 59ff267f387fc..7fcb6d7868ae1 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/abilities/source/WatermarkPushDownSpec.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/abilities/source/WatermarkPushDownSpec.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier; import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.configuration.Configuration; import org.apache.flink.table.api.TableException; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.connector.source.abilities.SupportsWatermarkPushDown; @@ -79,11 +78,9 @@ public void apply(DynamicTableSource tableSource, SourceAbilityContext context) context.getSourceRowType(), watermarkExpr, Option.apply("context")); - Configuration configuration = context.getTableConfig().getConfiguration(); WatermarkGeneratorSupplier supplier = - new GeneratedWatermarkGeneratorSupplier( - configuration, generatedWatermarkGenerator); + new GeneratedWatermarkGeneratorSupplier(generatedWatermarkGenerator); WatermarkStrategy watermarkStrategy = WatermarkStrategy.forGenerator(supplier); if (idleTimeoutMillis > 0) { diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNode.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNode.java index 81d07b8d51001..d2b06fd534248 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNode.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNode.java @@ -50,6 +50,7 @@ public interface ExecNode extends ExecNodeTranslator { String FIELD_NAME_ID = "id"; String FIELD_NAME_TYPE = "type"; + String FIELD_NAME_CONFIGURATION = "configuration"; String FIELD_NAME_DESCRIPTION = "description"; String FIELD_NAME_INPUT_PROPERTIES = "inputProperties"; String FIELD_NAME_OUTPUT_TYPE = "outputType"; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java index 337230c8c9cfb..06b88507b3fca 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java @@ -24,6 +24,7 @@ import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.delegation.Planner; import org.apache.flink.table.planner.delegation.PlannerBase; +import org.apache.flink.table.planner.plan.nodes.exec.serde.ConfigurationJsonSerializerFilter; import org.apache.flink.table.planner.plan.nodes.exec.utils.TransformationMetadata; import org.apache.flink.table.planner.plan.nodes.exec.visitor.ExecNodeVisitor; import org.apache.flink.table.planner.plan.utils.ExecNodeMetadataUtil; @@ -31,6 +32,7 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonInclude; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; import java.util.ArrayList; @@ -70,13 +72,30 @@ protected final ExecNodeContext getContextFromAnnotation() { return ExecNodeContext.newContext(this.getClass()).withId(getId()); } + @JsonProperty(value = FIELD_NAME_CONFIGURATION, access = JsonProperty.Access.WRITE_ONLY) + private final ReadableConfig persistedConfig; + + @JsonProperty( + value = FIELD_NAME_CONFIGURATION, + access = JsonProperty.Access.READ_ONLY, + index = 2) + // Custom filter to exclude node configuration if no consumed options are used + @JsonInclude( + value = JsonInclude.Include.CUSTOM, + valueFilter = ConfigurationJsonSerializerFilter.class) + public ReadableConfig getPersistedConfig() { + return persistedConfig; + } + protected ExecNodeBase( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List inputProperties, LogicalType outputType, String description) { this.context = checkNotNull(context).withId(id); + this.persistedConfig = persistedConfig == null ? new Configuration() : persistedConfig; this.inputProperties = checkNotNull(inputProperties); this.outputType = checkNotNull(outputType); this.description = checkNotNull(description); @@ -129,9 +148,7 @@ public final Transformation translateToPlan(Planner planner) { translateToPlanInternal( (PlannerBase) planner, new ExecNodeConfig( - ((PlannerBase) planner).getConfiguration(), - ((PlannerBase) planner).getTableConfig(), - new Configuration())); + ((PlannerBase) planner).getTableConfig(), persistedConfig)); if (this instanceof SingleTransformationTranslator) { if (inputsContainSingleton()) { transformation.setParallelism(1); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java index 183ce14aeb232..ebfe752141b60 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java @@ -20,59 +20,36 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.config.ExecutionConfigOptions; -import org.apache.flink.table.planner.codegen.CodeGeneratorContext; import org.apache.flink.table.planner.delegation.PlannerBase; -import org.apache.flink.table.planner.delegation.PlannerConfig; -import java.time.ZoneId; import java.util.Optional; /** - * Configuration view which is used combine the {@link PlannerConfig} with the {@link - * ExecNodeBase#getNodeConfig()} configuration. The persisted configuration of the {@link ExecNode} - * which is deserialized from the JSON plan has precedence over the {@link PlannerConfig}. + * Configuration view which is used combine the {@link PlannerBase#getTableConfig()} with the {@link + * ExecNodeBase#getPersistedConfig()} configuration. The persisted configuration of the {@link + * ExecNode} which is deserialized from the JSON plan has precedence over the {@link + * PlannerBase#getTableConfig()}. */ @Internal public final class ExecNodeConfig implements ReadableConfig { - private final ReadableConfig plannerConfig; - - // See https://issues.apache.org/jira/browse/FLINK-26190 - // Used only for the deprecated getMaxIdleStateRetentionTime to also satisfy tests which - // manipulate maxIdleStateRetentionTime, like OverAggregateHarnessTest. - private final TableConfig originalTableConfig; - // See https://issues.apache.org/jira/browse/FLINK-26190 private final TableConfig tableConfig; private final ReadableConfig nodeConfig; - ExecNodeConfig( - ReadableConfig plannerConfig, TableConfig tableConfig, ReadableConfig nodeConfig) { - this.plannerConfig = plannerConfig; + ExecNodeConfig(TableConfig tableConfig, ReadableConfig nodeConfig) { this.nodeConfig = nodeConfig; - this.originalTableConfig = tableConfig; - this.tableConfig = TableConfig.getDefault(); - this.tableConfig.setNullCheck(tableConfig.getNullCheck()); - this.tableConfig.setDecimalContext(tableConfig.getDecimalContext()); - this.tableConfig.addConfiguration(tableConfig.getConfiguration()); - this.tableConfig.addConfiguration((Configuration) nodeConfig); + this.tableConfig = tableConfig; } /** - * Return the merged {@link TableConfig} from {@link PlannerBase#getTableConfig()} and {@link - * ExecNodeBase#getNodeConfig()}. + * Return the {@link PlannerBase#getTableConfig()}. * - * @return the {@link TableConfig}. - * @deprecated This method is used only for {@link CodeGeneratorContext} and related methods, - * which end up passing the {@link TableConfig} to the {@link CodeGeneratorContext}. It - * should be removed once {@link CodeGeneratorContext#nullCheck()} is removed, since for all - * other usages it's possible to use the {@link ReadableConfig}. + * @return the {@link PlannerBase#getTableConfig()}. */ - // See https://issues.apache.org/jira/browse/FLINK-26190 @Deprecated public TableConfig getTableConfig() { return tableConfig; @@ -80,7 +57,7 @@ public TableConfig getTableConfig() { @Override public T get(ConfigOption option) { - return nodeConfig.getOptional(option).orElseGet(() -> plannerConfig.get(option)); + return nodeConfig.getOptional(option).orElseGet(() -> tableConfig.get(option)); } @Override @@ -89,28 +66,11 @@ public Optional getOptional(ConfigOption option) { if (tableValue.isPresent()) { return tableValue; } - return plannerConfig.getOptional(option); + return tableConfig.getOptional(option); } /** @return The duration until state which was not updated will be retained. */ public long getStateRetentionTime() { return get(ExecutionConfigOptions.IDLE_STATE_RETENTION).toMillis(); } - - // See https://issues.apache.org/jira/browse/FLINK-26190 - /** - * Using {@link #originalTableConfig} to satisify tests like {@code OverAggregateHarnessTest}, - * which use {@code HarnessTestBase#TestTableConfig} to individually manipulate the - * maxIdleStateRetentionTime. See {@link TableConfig#getMaxIdleStateRetentionTime()}. - */ - @Deprecated - public long getMaxIdleStateRetentionTime() { - return originalTableConfig.getMaxIdleStateRetentionTime(); - } - - // See https://issues.apache.org/jira/browse/FLINK-26190 - /** See {@link TableConfig#getLocalTimeZone()}. */ - public ZoneId getLocalTimeZone() { - return tableConfig.getLocalTimeZone(); - } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeContext.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeContext.java index d00d681ac4c15..0abb680b13655 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeContext.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeContext.java @@ -20,6 +20,8 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.planner.plan.utils.ExecNodeMetadataUtil; import org.apache.flink.table.types.logical.LogicalType; @@ -33,6 +35,7 @@ import java.util.List; import java.util.concurrent.atomic.AtomicInteger; import java.util.regex.Pattern; +import java.util.stream.Stream; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -81,10 +84,11 @@ private ExecNodeContext(String name, Integer version) { * @param id The unique id of the {@link ExecNode}. See {@link ExecNode#getId()}. It can be null * initially and then later set by using {@link #withId(int)} which creates a new instance * of {@link ExecNodeContext} since it's immutable. This way we can satisfy both the {@link - * ExecNodeBase#ExecNodeBase(int, ExecNodeContext, List, LogicalType, String)} ctor, which - * is used for the {@link JsonCreator} ctors, where the {@code id} and the {@code context} - * are read separately, and the {@link ExecNodeBase#getContextFromAnnotation()} which - * creates a new context with a new id provided by: {@link #newNodeId()}. + * ExecNodeBase#ExecNodeBase(int, ExecNodeContext, ReadableConfig, List, LogicalType, + * String)} ctor, which is used for the {@link JsonCreator} ctors, where the {@code id} and + * the {@code context} are read separately, and the {@link + * ExecNodeBase#getContextFromAnnotation()} which creates a new context with a new id + * provided by: {@link #newNodeId()}. * @param name The name of the {@link ExecNode}. See {@link ExecNodeMetadata#name()}. * @param version The version of the {@link ExecNode}. See {@link ExecNodeMetadata#version()}. */ @@ -176,4 +180,22 @@ public static > ExecNodeContext newContext(Class execNo } return new ExecNodeContext(metadata.name(), metadata.version()); } + + /** + * Create a configuration for the {@link ExecNode}, ready to be persisted to a JSON plan. + * + * @param execNodeClass The {@link ExecNode} class. + * @param tableConfig The planner configuration (include the {@link TableConfig}). + * @return The {@link ExecNode} configuration, which contains the consumed options for the node, + * defined by {@link ExecNodeMetadata#consumedOptions()}, along with their values. + */ + public static > ReadableConfig newPersistedConfig( + Class execNodeClass, ReadableConfig tableConfig) { + return ExecNodeMetadataUtil.newPersistedConfig( + execNodeClass, + tableConfig, + Stream.concat( + ExecNodeMetadataUtil.TABLE_CONFIG_OPTIONS.stream(), + ExecNodeMetadataUtil.EXECUTION_CONFIG_OPTIONS.stream())); + } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecBoundedStreamScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecBoundedStreamScan.java index 167870e5f2c58..fb8a21752ceab 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecBoundedStreamScan.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecBoundedStreamScan.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.CodeGeneratorContext; @@ -48,6 +49,7 @@ public class BatchExecBoundedStreamScan extends ExecNodeBase private final List qualifiedName; public BatchExecBoundedStreamScan( + ReadableConfig tableConfig, DataStream dataStream, DataType sourceType, int[] fieldIndexes, @@ -57,6 +59,7 @@ public BatchExecBoundedStreamScan( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecBoundedStreamScan.class), + ExecNodeContext.newPersistedConfig(BatchExecBoundedStreamScan.class, tableConfig), Collections.emptyList(), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecCalc.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecCalc.java index a7063afcd9769..f2f3f5b6b3676 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecCalc.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecCalc.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -37,6 +38,7 @@ public class BatchExecCalc extends CommonExecCalc implements BatchExecNode { public BatchExecCalc( + ReadableConfig tableConfig, List projection, @Nullable RexNode condition, InputProperty inputProperty, @@ -45,6 +47,7 @@ public BatchExecCalc( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecCalc.class), + ExecNodeContext.newPersistedConfig(BatchExecCalc.class, tableConfig), projection, condition, TableStreamOperator.class, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecCorrelate.java index c8e538b8ddd40..3b5450c094103 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecCorrelate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecCorrelate.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; import org.apache.flink.table.planner.plan.nodes.exec.InputProperty; @@ -37,6 +38,7 @@ public class BatchExecCorrelate extends CommonExecCorrelate implements BatchExecNode { public BatchExecCorrelate( + ReadableConfig tableConfig, FlinkJoinType joinType, RexCall invocation, @Nullable RexNode condition, @@ -46,6 +48,7 @@ public BatchExecCorrelate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecCorrelate.class), + ExecNodeContext.newPersistedConfig(BatchExecCorrelate.class, tableConfig), joinType, invocation, condition, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExchange.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExchange.java index c07bd85d5479e..339dc8f40ffad 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExchange.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExchange.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.transformations.PartitionTransformation; import org.apache.flink.streaming.api.transformations.StreamExchangeMode; import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner; @@ -66,10 +67,15 @@ public class BatchExecExchange extends CommonExecExchange implements BatchExecNo // if it's None, use value from configuration @Nullable private StreamExchangeMode requiredExchangeMode; - public BatchExecExchange(InputProperty inputProperty, RowType outputType, String description) { + public BatchExecExchange( + ReadableConfig tableConfig, + InputProperty inputProperty, + RowType outputType, + String description) { super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecExchange.class), + ExecNodeContext.newPersistedConfig(BatchExecExchange.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExpand.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExpand.java index 653f1d7e19fcd..c91e21c1da042 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExpand.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExpand.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -34,6 +35,7 @@ public class BatchExecExpand extends CommonExecExpand implements BatchExecNode { public BatchExecExpand( + ReadableConfig tableConfig, List> projects, InputProperty inputProperty, RowType outputType, @@ -41,6 +43,7 @@ public BatchExecExpand( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecExpand.class), + ExecNodeContext.newPersistedConfig(BatchExecExpand.class, tableConfig), projects, false, // retainHeader Collections.singletonList(inputProperty), diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java index 98511a22300fd..59a6fc65958f9 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.data.RowData; @@ -59,6 +60,7 @@ public class BatchExecHashAggregate extends ExecNodeBase private final boolean isFinal; public BatchExecHashAggregate( + ReadableConfig tableConfig, int[] grouping, int[] auxGrouping, AggregateCall[] aggCalls, @@ -71,6 +73,7 @@ public BatchExecHashAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecHashAggregate.class), + ExecNodeContext.newPersistedConfig(BatchExecHashAggregate.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java index e003eb22bd8e7..b6ee3a1a14021 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.streaming.api.operators.StreamOperatorFactory; import org.apache.flink.table.api.config.ExecutionConfigOptions; @@ -61,6 +62,7 @@ public class BatchExecHashJoin extends ExecNodeBase private final boolean tryDistinctBuildRow; public BatchExecHashJoin( + ReadableConfig tableConfig, JoinSpec joinSpec, int estimatedLeftAvgRowSize, int estimatedRightAvgRowSize, @@ -75,6 +77,7 @@ public BatchExecHashJoin( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecHashJoin.class), + ExecNodeContext.newPersistedConfig(BatchExecHashJoin.class, tableConfig), Arrays.asList(leftInputProperty, rightInputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashWindowAggregate.java index c16f386b4173e..d43c0cddd16e4 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashWindowAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashWindowAggregate.java @@ -20,6 +20,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.data.RowData; @@ -67,6 +68,7 @@ public class BatchExecHashWindowAggregate extends ExecNodeBase private final boolean isFinal; public BatchExecHashWindowAggregate( + ReadableConfig tableConfig, int[] grouping, int[] auxGrouping, AggregateCall[] aggCalls, @@ -84,6 +86,7 @@ public BatchExecHashWindowAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecHashWindowAggregate.class), + ExecNodeContext.newPersistedConfig(BatchExecHashWindowAggregate.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacySink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacySink.java index 4440d8725c279..c478366028051 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacySink.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacySink.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -41,6 +42,7 @@ public class BatchExecLegacySink extends CommonExecLegacySink implements BatchExecNode { public BatchExecLegacySink( + ReadableConfig tableConfig, TableSink tableSink, @Nullable String[] upsertKeys, InputProperty inputProperty, @@ -49,6 +51,7 @@ public BatchExecLegacySink( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecLegacySink.class), + ExecNodeContext.newPersistedConfig(BatchExecLegacySink.class, tableConfig), tableSink, upsertKeys, false, // needRetraction diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacyTableSourceScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacyTableSourceScan.java index 69a728b52fe57..5696fd48001ea 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacyTableSourceScan.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacyTableSourceScan.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.io.InputSplit; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction; @@ -55,6 +56,7 @@ public class BatchExecLegacyTableSourceScan extends CommonExecLegacyTableSourceS implements BatchExecNode { public BatchExecLegacyTableSourceScan( + ReadableConfig tableConfig, TableSource tableSource, List qualifiedName, RowType outputType, @@ -62,6 +64,8 @@ public BatchExecLegacyTableSourceScan( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecLegacyTableSourceScan.class), + ExecNodeContext.newPersistedConfig( + BatchExecLegacyTableSourceScan.class, tableConfig), tableSource, qualifiedName, outputType, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLimit.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLimit.java index 001001c150ae4..17844d6617b29 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLimit.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLimit.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; @@ -41,6 +42,7 @@ public class BatchExecLimit extends ExecNodeBase implements BatchExecNo private final boolean isGlobal; public BatchExecLimit( + ReadableConfig tableConfig, long limitStart, long limitEnd, boolean isGlobal, @@ -50,6 +52,7 @@ public BatchExecLimit( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecLimit.class), + ExecNodeContext.newPersistedConfig(BatchExecLimit.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLookupJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLookupJoin.java index f756c2ea269ad..c9f9180941fc6 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLookupJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLookupJoin.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; import org.apache.flink.table.planner.plan.nodes.exec.InputProperty; @@ -38,6 +39,7 @@ /** {@link BatchExecNode} for temporal table join that implemented by lookup. */ public class BatchExecLookupJoin extends CommonExecLookupJoin implements BatchExecNode { public BatchExecLookupJoin( + ReadableConfig tableConfig, FlinkJoinType joinType, @Nullable RexNode joinCondition, TemporalTableSourceSpec temporalTableSourceSpec, @@ -50,6 +52,7 @@ public BatchExecLookupJoin( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecLookupJoin.class), + ExecNodeContext.newPersistedConfig(BatchExecLookupJoin.class, tableConfig), joinType, joinCondition, temporalTableSourceSpec, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecMultipleInput.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecMultipleInput.java index 15fd0498d7686..1420610190f15 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecMultipleInput.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecMultipleInput.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.transformations.MultipleInputTransformation; import org.apache.flink.table.data.RowData; @@ -78,6 +79,7 @@ public class BatchExecMultipleInput extends ExecNodeBase private final List originalEdges; public BatchExecMultipleInput( + ReadableConfig tableConfig, List inputProperties, ExecNode rootNode, List originalEdges, @@ -85,6 +87,7 @@ public BatchExecMultipleInput( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecMultipleInput.class), + ExecNodeContext.newPersistedConfig(BatchExecMultipleInput.class, tableConfig), inputProperties, rootNode.getOutputType(), description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecNestedLoopJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecNestedLoopJoin.java index 0d868ff353684..9b5401fdc4a66 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecNestedLoopJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecNestedLoopJoin.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.CodeGeneratorContext; @@ -51,6 +52,7 @@ public class BatchExecNestedLoopJoin extends ExecNodeBase private final boolean singleRowJoin; public BatchExecNestedLoopJoin( + ReadableConfig tableConfig, FlinkJoinType joinType, RexNode condition, boolean leftIsBuild, @@ -62,6 +64,7 @@ public BatchExecNestedLoopJoin( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecNestedLoopJoin.class), + ExecNodeContext.newPersistedConfig(BatchExecNestedLoopJoin.class, tableConfig), Arrays.asList(leftInputProperty, rightInputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregate.java index 6e4805ce5a158..357e28dda85f9 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregate.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.config.ExecutionConfigOptions; @@ -75,6 +76,7 @@ public class BatchExecOverAggregate extends BatchExecOverAggregateBase { public BatchExecOverAggregate( + ReadableConfig tableConfig, OverSpec overSpec, InputProperty inputProperty, RowType outputType, @@ -82,6 +84,7 @@ public BatchExecOverAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecOverAggregate.class), + ExecNodeContext.newPersistedConfig(BatchExecOverAggregate.class, tableConfig), overSpec, inputProperty, outputType, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregateBase.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregateBase.java index 3121d9d2c2132..860d40b34e14a 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregateBase.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregateBase.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; @@ -47,11 +48,18 @@ public abstract class BatchExecOverAggregateBase extends ExecNodeBase public BatchExecOverAggregateBase( int id, ExecNodeContext context, + ReadableConfig persistedConfig, OverSpec overSpec, InputProperty inputProperty, RowType outputType, String description) { - super(id, context, Collections.singletonList(inputProperty), outputType, description); + super( + id, + context, + persistedConfig, + Collections.singletonList(inputProperty), + outputType, + description); this.overSpec = overSpec; } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonCalc.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonCalc.java index 9c6c59876ae00..7ecd93ac0df21 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonCalc.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonCalc.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -34,6 +35,7 @@ public class BatchExecPythonCalc extends CommonExecPythonCalc implements BatchExecNode { public BatchExecPythonCalc( + ReadableConfig tableConfig, List projection, InputProperty inputProperty, RowType outputType, @@ -41,6 +43,7 @@ public BatchExecPythonCalc( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecPythonCalc.class), + ExecNodeContext.newPersistedConfig(BatchExecPythonCalc.class, tableConfig), projection, Collections.singletonList(inputProperty), outputType, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonCorrelate.java index 8307f6f27444e..0d58251a80092 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonCorrelate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonCorrelate.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; import org.apache.flink.table.planner.plan.nodes.exec.InputProperty; @@ -34,6 +35,7 @@ public class BatchExecPythonCorrelate extends CommonExecPythonCorrelate implements BatchExecNode { public BatchExecPythonCorrelate( + ReadableConfig tableConfig, FlinkJoinType joinType, RexCall invocation, InputProperty inputProperty, @@ -42,6 +44,7 @@ public BatchExecPythonCorrelate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecPythonCorrelate.class), + ExecNodeContext.newPersistedConfig(BatchExecPythonCorrelate.class, tableConfig), joinType, invocation, Collections.singletonList(inputProperty), diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupAggregate.java index 46ada1ba62867..dc13cac46db64 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupAggregate.java @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -63,6 +64,7 @@ public class BatchExecPythonGroupAggregate extends ExecNodeBase private final AggregateCall[] aggCalls; public BatchExecPythonGroupAggregate( + ReadableConfig tableConfig, int[] grouping, int[] auxGrouping, AggregateCall[] aggCalls, @@ -72,6 +74,8 @@ public BatchExecPythonGroupAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecPythonGroupAggregate.class), + ExecNodeContext.newPersistedConfig( + BatchExecPythonGroupAggregate.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); @@ -90,7 +94,7 @@ protected Transformation translateToPlanInternal( final RowType inputRowType = (RowType) inputEdge.getOutputType(); final RowType outputRowType = InternalTypeInfo.of(getOutputType()).toRowType(); Configuration pythonConfig = - CommonPythonUtil.getMergedConfig(planner.getExecEnv(), config.getTableConfig()); + CommonPythonUtil.extractPythonConfiguration(planner.getExecEnv(), config); OneInputTransformation transform = createPythonOneInputTransformation( inputTransform, inputRowType, outputRowType, pythonConfig, config); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupWindowAggregate.java index 1833970894f7b..2590673126ac6 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupWindowAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupWindowAggregate.java @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -75,6 +76,7 @@ public class BatchExecPythonGroupWindowAggregate extends ExecNodeBase private final NamedWindowProperty[] namedWindowProperties; public BatchExecPythonGroupWindowAggregate( + ReadableConfig tableConfig, int[] grouping, int[] auxGrouping, AggregateCall[] aggCalls, @@ -87,6 +89,8 @@ public BatchExecPythonGroupWindowAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecPythonGroupWindowAggregate.class), + ExecNodeContext.newPersistedConfig( + BatchExecPythonGroupWindowAggregate.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); @@ -110,7 +114,7 @@ protected Transformation translateToPlanInternal( final Tuple2 windowSizeAndSlideSize = WindowCodeGenerator.getWindowDef(window); final Configuration pythonConfig = - CommonPythonUtil.getMergedConfig(planner.getExecEnv(), config.getTableConfig()); + CommonPythonUtil.extractPythonConfiguration(planner.getExecEnv(), config); int groupBufferLimitSize = pythonConfig.getInteger( ExecutionConfigOptions.TABLE_EXEC_WINDOW_AGG_BUFFER_SIZE_LIMIT); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonOverAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonOverAggregate.java index c9529ad3d1231..4b347bc785362 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonOverAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonOverAggregate.java @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -69,6 +70,7 @@ public class BatchExecPythonOverAggregate extends BatchExecOverAggregateBase { private final List aggWindowIndex; public BatchExecPythonOverAggregate( + ReadableConfig tableConfig, OverSpec overSpec, InputProperty inputProperty, RowType outputType, @@ -76,6 +78,7 @@ public BatchExecPythonOverAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecPythonOverAggregate.class), + ExecNodeContext.newPersistedConfig(BatchExecPythonOverAggregate.class, tableConfig), overSpec, inputProperty, outputType, @@ -150,7 +153,7 @@ protected Transformation translateToPlanInternal( } } Configuration pythonConfig = - CommonPythonUtil.getMergedConfig(planner.getExecEnv(), config.getTableConfig()); + CommonPythonUtil.extractPythonConfiguration(planner.getExecEnv(), config); OneInputTransformation transform = createPythonOneInputTransformation( inputTransform, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecRank.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecRank.java index 0a86211687e00..daade9c6dab1d 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecRank.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecRank.java @@ -20,6 +20,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.sort.ComparatorCodeGenerator; @@ -51,6 +52,7 @@ public class BatchExecRank extends ExecNodeBase implements InputSortedE private final boolean outputRankNumber; public BatchExecRank( + ReadableConfig tableConfig, int[] partitionFields, int[] sortFields, long rankStart, @@ -62,6 +64,7 @@ public BatchExecRank( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecRank.class), + ExecNodeContext.newPersistedConfig(BatchExecRank.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSink.java index d9c5b35d04fa4..da1637f962096 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSink.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSink.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.data.RowData; @@ -39,6 +40,7 @@ */ public class BatchExecSink extends CommonExecSink implements BatchExecNode { public BatchExecSink( + ReadableConfig tableConfig, DynamicTableSinkSpec tableSinkSpec, InputProperty inputProperty, LogicalType outputType, @@ -46,6 +48,7 @@ public BatchExecSink( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecSink.class), + ExecNodeContext.newPersistedConfig(BatchExecSink.class, tableConfig), tableSinkSpec, ChangelogMode.insertOnly(), true, // isBounded diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSort.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSort.java index 8f8e42082d9a5..44f1e19db4926 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSort.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSort.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.data.RowData; @@ -47,6 +48,7 @@ public class BatchExecSort extends ExecNodeBase implements BatchExecNod private final SortSpec sortSpec; public BatchExecSort( + ReadableConfig tableConfig, SortSpec sortSpec, InputProperty inputProperty, RowType outputType, @@ -54,6 +56,7 @@ public BatchExecSort( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecSort.class), + ExecNodeContext.newPersistedConfig(BatchExecSort.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortAggregate.java index e550fcec7f5e3..4daf095667be1 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortAggregate.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.CodeGeneratorContext; @@ -58,6 +59,7 @@ public class BatchExecSortAggregate extends ExecNodeBase private final boolean isFinal; public BatchExecSortAggregate( + ReadableConfig tableConfig, int[] grouping, int[] auxGrouping, AggregateCall[] aggCalls, @@ -70,6 +72,7 @@ public BatchExecSortAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecSortAggregate.class), + ExecNodeContext.newPersistedConfig(BatchExecSortAggregate.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortLimit.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortLimit.java index 3cbaefbc61e96..a2fdf205adb69 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortLimit.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortLimit.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.RowData; @@ -53,6 +54,7 @@ public class BatchExecSortLimit extends ExecNodeBase private final boolean isGlobal; public BatchExecSortLimit( + ReadableConfig tableConfig, SortSpec sortSpec, long limitStart, long limitEnd, @@ -63,6 +65,7 @@ public BatchExecSortLimit( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecSortLimit.class), + ExecNodeContext.newPersistedConfig(BatchExecSortLimit.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortMergeJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortMergeJoin.java index 9d8e7d7327520..b38d3264be736 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortMergeJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortMergeJoin.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.data.RowData; @@ -65,6 +66,7 @@ public class BatchExecSortMergeJoin extends ExecNodeBase private final boolean leftIsSmaller; public BatchExecSortMergeJoin( + ReadableConfig tableConfig, FlinkJoinType joinType, int[] leftKeys, int[] rightKeys, @@ -78,6 +80,7 @@ public BatchExecSortMergeJoin( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecSortMergeJoin.class), + ExecNodeContext.newPersistedConfig(BatchExecSortMergeJoin.class, tableConfig), Arrays.asList(leftInputProperty, rightInputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortWindowAggregate.java index d91a4515e75f8..ce95fb9e0259f 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortWindowAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortWindowAggregate.java @@ -20,6 +20,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.data.RowData; @@ -67,6 +68,7 @@ public class BatchExecSortWindowAggregate extends ExecNodeBase private final boolean isFinal; public BatchExecSortWindowAggregate( + ReadableConfig tableConfig, int[] grouping, int[] auxGrouping, AggregateCall[] aggCalls, @@ -84,6 +86,7 @@ public BatchExecSortWindowAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecSortWindowAggregate.class), + ExecNodeContext.newPersistedConfig(BatchExecSortWindowAggregate.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecTableSourceScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecTableSourceScan.java index bfb58159d6337..95e98273fb9ce 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecTableSourceScan.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecTableSourceScan.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction; import org.apache.flink.table.connector.source.ScanTableSource; @@ -42,10 +43,14 @@ public class BatchExecTableSourceScan extends CommonExecTableSourceScan implements BatchExecNode { public BatchExecTableSourceScan( - DynamicTableSourceSpec tableSourceSpec, RowType outputType, String description) { + ReadableConfig tableConfig, + DynamicTableSourceSpec tableSourceSpec, + RowType outputType, + String description) { super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecTableSourceScan.class), + ExecNodeContext.newPersistedConfig(BatchExecTableSourceScan.class, tableConfig), tableSourceSpec, outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecUnion.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecUnion.java index 506243b060496..d365206b356f0 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecUnion.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecUnion.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -34,10 +35,14 @@ public class BatchExecUnion extends CommonExecUnion implements BatchExecNode { public BatchExecUnion( - List inputProperties, RowType outputType, String description) { + ReadableConfig tableConfig, + List inputProperties, + RowType outputType, + String description) { super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecUnion.class), + ExecNodeContext.newPersistedConfig(BatchExecUnion.class, tableConfig), inputProperties, outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecValues.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecValues.java index ab8700f44534b..b1be0f8395488 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecValues.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecValues.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; @@ -35,10 +36,15 @@ /** Batch {@link ExecNode} that read records from given values. */ public class BatchExecValues extends CommonExecValues implements BatchExecNode { - public BatchExecValues(List> tuples, RowType outputType, String description) { + public BatchExecValues( + ReadableConfig tableConfig, + List> tuples, + RowType outputType, + String description) { super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecValues.class), + ExecNodeContext.newPersistedConfig(BatchExecValues.class, tableConfig), tuples, outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecWindowTableFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecWindowTableFunction.java index 1e36ed1778f94..b00ce302c1940 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecWindowTableFunction.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecWindowTableFunction.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; @@ -37,6 +38,7 @@ public class BatchExecWindowTableFunction extends CommonExecWindowTableFunction implements BatchExecNode { public BatchExecWindowTableFunction( + ReadableConfig tableConfig, TimeAttributeWindowingStrategy windowingStrategy, InputProperty inputProperty, RowType outputType, @@ -44,6 +46,7 @@ public BatchExecWindowTableFunction( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecWindowTableFunction.class), + ExecNodeContext.newPersistedConfig(BatchExecWindowTableFunction.class, tableConfig), windowingStrategy, Collections.singletonList(inputProperty), outputType, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCalc.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCalc.java index 24024c6e10011..b954afae062c2 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCalc.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCalc.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.common; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.CalcCodeGenerator; import org.apache.flink.table.planner.codegen.CodeGeneratorContext; @@ -68,6 +69,7 @@ public abstract class CommonExecCalc extends ExecNodeBase protected CommonExecCalc( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List projection, @Nullable RexNode condition, Class operatorBaseClass, @@ -75,7 +77,7 @@ protected CommonExecCalc( List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.projection = checkNotNull(projection); this.condition = condition; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCorrelate.java index bab655c69ea6e..e75bd2b3d33a0 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCorrelate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCorrelate.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.common; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.CodeGeneratorContext; import org.apache.flink.table.planner.codegen.CorrelateCodeGenerator; @@ -72,6 +73,7 @@ public abstract class CommonExecCorrelate extends ExecNodeBase public CommonExecCorrelate( int id, ExecNodeContext context, + ReadableConfig persistedConfig, FlinkJoinType joinType, RexCall invocation, @Nullable RexNode condition, @@ -80,7 +82,7 @@ public CommonExecCorrelate( List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.joinType = checkNotNull(joinType); this.invocation = checkNotNull(invocation); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExchange.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExchange.java index f19498acecd74..809bcf94f27ae 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExchange.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExchange.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.common; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -38,9 +39,10 @@ public abstract class CommonExecExchange extends ExecNodeBase public CommonExecExchange( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExpand.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExpand.java index ddb6040ee33f1..a939a06934be5 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExpand.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExpand.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.common; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.CodeGeneratorContext; import org.apache.flink.table.planner.codegen.ExpandCodeGenerator; @@ -60,12 +61,13 @@ public abstract class CommonExecExpand extends ExecNodeBase public CommonExecExpand( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List> projects, boolean retainHeader, List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.projects = checkNotNull(projects); checkArgument( diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacySink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacySink.java index ea72482dc86d9..8f731f6ba5412 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacySink.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacySink.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.table.api.TableException; @@ -70,6 +71,7 @@ public abstract class CommonExecLegacySink extends ExecNodeBase public CommonExecLegacySink( int id, ExecNodeContext context, + ReadableConfig persistedConfig, TableSink tableSink, @Nullable String[] upsertKeys, boolean needRetraction, @@ -77,7 +79,13 @@ public CommonExecLegacySink( InputProperty inputProperty, LogicalType outputType, String description) { - super(id, context, Collections.singletonList(inputProperty), outputType, description); + super( + id, + context, + persistedConfig, + Collections.singletonList(inputProperty), + outputType, + description); this.tableSink = tableSink; this.upsertKeys = upsertKeys; this.needRetraction = needRetraction; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacyTableSourceScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacyTableSourceScan.java index 0c57259acce21..5769d8ad86a56 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacyTableSourceScan.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacyTableSourceScan.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.io.InputSplit; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableException; @@ -71,11 +72,12 @@ public abstract class CommonExecLegacyTableSourceScan extends ExecNodeBase tableSource, List qualifiedName, RowType outputType, String description) { - super(id, context, Collections.emptyList(), outputType, description); + super(id, context, persistedConfig, Collections.emptyList(), outputType, description); this.tableSource = tableSource; this.qualifiedName = qualifiedName; } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLookupJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLookupJoin.java index c50deb92545a6..1986507cb21b1 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLookupJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLookupJoin.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.AsyncDataStream; import org.apache.flink.streaming.api.functions.ProcessFunction; import org.apache.flink.streaming.api.functions.async.AsyncFunction; @@ -176,6 +177,7 @@ public abstract class CommonExecLookupJoin extends ExecNodeBase protected CommonExecLookupJoin( int id, ExecNodeContext context, + ReadableConfig persistedConfig, FlinkJoinType joinType, @Nullable RexNode joinCondition, // TODO: refactor this into TableSourceTable, once legacy TableSource is removed @@ -186,7 +188,7 @@ protected CommonExecLookupJoin( List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.joinType = checkNotNull(joinType); this.joinCondition = joinCondition; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCalc.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCalc.java index 6d63f3b852fb4..16cd36cade475 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCalc.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCalc.java @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -84,11 +85,12 @@ public abstract class CommonExecPythonCalc extends ExecNodeBase public CommonExecPythonCalc( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List projection, List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.projection = checkNotNull(projection); } @@ -101,7 +103,7 @@ protected Transformation translateToPlanInternal( final Transformation inputTransform = (Transformation) inputEdge.translateToPlan(planner); final Configuration pythonConfig = - CommonPythonUtil.getMergedConfig(planner.getExecEnv(), config.getTableConfig()); + CommonPythonUtil.extractPythonConfiguration(planner.getExecEnv(), config); OneInputTransformation ret = createPythonOneInputTransformation(inputTransform, config, pythonConfig); if (CommonPythonUtil.isPythonWorkerUsingManagedMemory(pythonConfig)) { diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCorrelate.java index 9af214c5a20b6..a9aae7ad34315 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCorrelate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCorrelate.java @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -69,12 +70,13 @@ public abstract class CommonExecPythonCorrelate extends ExecNodeBase public CommonExecPythonCorrelate( int id, ExecNodeContext context, + ReadableConfig persistedConfig, FlinkJoinType joinType, RexCall invocation, List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.joinType = joinType; this.invocation = invocation; @@ -88,7 +90,7 @@ protected Transformation translateToPlanInternal( final Transformation inputTransform = (Transformation) inputEdge.translateToPlan(planner); final Configuration pythonConfig = - CommonPythonUtil.getMergedConfig(planner.getExecEnv(), config.getTableConfig()); + CommonPythonUtil.extractPythonConfiguration(planner.getExecEnv(), config); OneInputTransformation transform = createPythonOneInputTransformation(inputTransform, config, pythonConfig); if (CommonPythonUtil.isPythonWorkerUsingManagedMemory(pythonConfig)) { diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSink.java index ce657c9fec14a..00cf4daea7368 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSink.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSink.java @@ -23,6 +23,7 @@ import org.apache.flink.api.java.typeutils.InputTypeConfigurable; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.runtime.state.KeyGroupRangeAssignment; +import org.apache.flink.streaming.api.datastream.CustomSinkOperatorUidHashes; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -109,13 +110,14 @@ public abstract class CommonExecSink extends ExecNodeBase protected CommonExecSink( int id, ExecNodeContext context, + ReadableConfig persistedConfig, DynamicTableSinkSpec tableSinkSpec, ChangelogMode inputChangelogMode, boolean isBounded, List inputProperties, LogicalType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); this.tableSinkSpec = tableSinkSpec; this.inputChangelogMode = inputChangelogMode; this.isBounded = isBounded; @@ -491,7 +493,9 @@ public Optional generateUid(String name) { final DataStream dataStream = new DataStream<>(env, sinkTransformation); final Transformation transformation = DataStreamSink.forSinkV1( - dataStream, ((SinkProvider) runtimeProvider).createSink()) + dataStream, + ((SinkProvider) runtimeProvider).createSink(), + CustomSinkOperatorUidHashes.DEFAULT) .getTransformation(); transformation.setParallelism(sinkParallelism); sinkMeta.fill(transformation); @@ -503,7 +507,9 @@ public Optional generateUid(String name) { final DataStream dataStream = new DataStream<>(env, sinkTransformation); final Transformation transformation = DataStreamSink.forSink( - dataStream, ((SinkV2Provider) runtimeProvider).createSink()) + dataStream, + ((SinkV2Provider) runtimeProvider).createSink(), + CustomSinkOperatorUidHashes.DEFAULT) .getTransformation(); transformation.setParallelism(sinkParallelism); sinkMeta.fill(transformation); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecTableSourceScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecTableSourceScan.java index 8d335407c3a7e..07e193b81e5bb 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecTableSourceScan.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecTableSourceScan.java @@ -74,10 +74,11 @@ public abstract class CommonExecTableSourceScan extends ExecNodeBase protected CommonExecTableSourceScan( int id, ExecNodeContext context, + ReadableConfig persistedConfig, DynamicTableSourceSpec tableSourceSpec, LogicalType outputType, String description) { - super(id, context, Collections.emptyList(), outputType, description); + super(id, context, persistedConfig, Collections.emptyList(), outputType, description); this.tableSourceSpec = tableSourceSpec; } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecUnion.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecUnion.java index 329475906b435..dddf6c6da8944 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecUnion.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecUnion.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.common; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.transformations.UnionTransformation; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; @@ -44,10 +45,11 @@ public abstract class CommonExecUnion extends ExecNodeBase public CommonExecUnion( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); } @SuppressWarnings({"unchecked", "rawtypes"}) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecValues.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecValues.java index 21be27f880fff..7e4639bb6e3cd 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecValues.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecValues.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.common; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.ValuesCodeGenerator; import org.apache.flink.table.planner.delegation.PlannerBase; @@ -50,10 +51,11 @@ public abstract class CommonExecValues extends ExecNodeBase public CommonExecValues( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List> tuples, RowType outputType, String description) { - super(id, context, Collections.emptyList(), outputType, description); + super(id, context, persistedConfig, Collections.emptyList(), outputType, description); this.tuples = tuples; } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecWindowTableFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecWindowTableFunction.java index fd46ec312753a..c39189e339bca 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecWindowTableFunction.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecWindowTableFunction.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.common; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; import org.apache.flink.table.planner.plan.logical.TimeAttributeWindowingStrategy; @@ -31,6 +32,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator; import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecNode; import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil; +import org.apache.flink.table.planner.utils.TableConfigUtils; import org.apache.flink.table.runtime.operators.window.TimeWindow; import org.apache.flink.table.runtime.operators.window.WindowTableFunctionOperator; import org.apache.flink.table.runtime.operators.window.assigners.WindowAssigner; @@ -61,11 +63,12 @@ public abstract class CommonExecWindowTableFunction extends ExecNodeBase inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.windowingStrategy = checkNotNull(windowingStrategy); } @@ -80,7 +83,8 @@ protected Transformation translateToPlanInternal( WindowAssigner windowAssigner = createWindowAssigner(windowingStrategy); final ZoneId shiftTimeZone = TimeWindowUtil.getShiftTimeZone( - windowingStrategy.getTimeAttributeType(), config.getLocalTimeZone()); + windowingStrategy.getTimeAttributeType(), + TableConfigUtils.getLocalTimeZone(config)); WindowTableFunctionOperator windowTableFunctionOperator = new WindowTableFunctionOperator( windowAssigner, windowingStrategy.getTimeAttributeIndex(), shiftTimeZone); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/DeadlockBreakupProcessor.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/DeadlockBreakupProcessor.java index 995926264a7fb..6c1e7695a328b 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/DeadlockBreakupProcessor.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/DeadlockBreakupProcessor.java @@ -44,7 +44,7 @@ public ExecNodeGraph process(ExecNodeGraph execGraph, ProcessorContext context) execGraph.getRootNodes(), InputProperty.DamBehavior.END_INPUT, StreamExchangeMode.BATCH, - context.getPlanner().getConfiguration()); + context.getPlanner().getTableConfig()); resolver.detectAndResolve(); return execGraph; } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/ForwardHashExchangeProcessor.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/ForwardHashExchangeProcessor.java index eae6279b901e3..9e03ac9da9687 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/ForwardHashExchangeProcessor.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/ForwardHashExchangeProcessor.java @@ -17,6 +17,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.processor; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; @@ -69,6 +70,7 @@ public ExecNodeGraph process(ExecNodeGraph execGraph, ProcessorContext context) if (!context.getPlanner().getExecEnv().getConfig().isDynamicGraph()) { return execGraph; } + ReadableConfig tableConfig = context.getPlanner().getTableConfig(); ExecNodeVisitor visitor = new AbstractExecNodeExactlyOnceVisitor() { @Override @@ -85,23 +87,25 @@ protected void visitNode(ExecNode node) { inputProperty.getRequiredDistribution(); ExecEdge edge = node.getInputEdges().get(i); - if (requiredDistribution.getType() == DistributionType.SINGLETON) { - if (!hasExchangeInput(edge) && isInputSortedNode(node)) { - // if operation chaining is disabled, this could mark sure the - // sort node and its output can also be connected by - // ForwardPartitioner + if (requiredDistribution.getType() != DistributionType.HASH) { + boolean visitChild = + requiredDistribution.getType() + == DistributionType.SINGLETON; + if (!hasExchangeInput(edge) + && hasSortInputForInputSortedNode(node)) { ExecEdge newEdge = - addExchangeAndReconnectEdge(edge, inputProperty, true); + addExchangeAndReconnectEdge( + tableConfig, + edge, + inputProperty, + true, + visitChild); newEdges.set(i, newEdge); changed = true; } continue; } - if (requiredDistribution.getType() != DistributionType.HASH) { - continue; - } - if (!hasExchangeInput(edge)) { ExecEdge newEdge; if (isInputSortedNode(node)) { @@ -111,9 +115,11 @@ protected void visitNode(ExecNode node) { ExecNode sort = edge.getSource(); ExecEdge newEdgeOfSort = addExchangeAndReconnectEdge( + tableConfig, sort.getInputEdges().get(0), inputProperty, - false); + false, + true); sort.setInputEdges( Collections.singletonList(newEdgeOfSort)); } @@ -122,12 +128,14 @@ protected void visitNode(ExecNode node) { // sort node and its output can also be connected by // ForwardPartitioner newEdge = - addExchangeAndReconnectEdge(edge, inputProperty, true); + addExchangeAndReconnectEdge( + tableConfig, edge, inputProperty, true, true); } else { // add Exchange with keep_input_as_is distribution as the input // of the node newEdge = - addExchangeAndReconnectEdge(edge, inputProperty, false); + addExchangeAndReconnectEdge( + tableConfig, edge, inputProperty, false, true); updateOriginalEdgeInMultipleInput( node, i, (BatchExecExchange) newEdge.getSource()); } @@ -138,7 +146,8 @@ protected void visitNode(ExecNode node) { // if operation chaining is disabled, this could mark sure the sort // node and its output can also be connected by ForwardPartitioner ExecEdge newEdge = - addExchangeAndReconnectEdge(edge, inputProperty, true); + addExchangeAndReconnectEdge( + tableConfig, edge, inputProperty, true, true); newEdges.set(i, newEdge); changed = true; } @@ -154,27 +163,36 @@ protected void visitNode(ExecNode node) { // TODO This implementation should be updated once FLINK-21224 is finished. private ExecEdge addExchangeAndReconnectEdge( - ExecEdge edge, InputProperty inputProperty, boolean strict) { + ReadableConfig tableConfig, + ExecEdge edge, + InputProperty inputProperty, + boolean strict, + boolean visitChild) { ExecNode target = edge.getTarget(); ExecNode source = edge.getSource(); if (source instanceof CommonExecExchange) { return edge; } // only Calc, Correlate and Sort can propagate sort property and distribution property - if (source instanceof BatchExecCalc - || source instanceof BatchExecPythonCalc - || source instanceof BatchExecSort - || source instanceof BatchExecCorrelate - || source instanceof BatchExecPythonCorrelate) { + if (visitChild + && (source instanceof BatchExecCalc + || source instanceof BatchExecPythonCalc + || source instanceof BatchExecSort + || source instanceof BatchExecCorrelate + || source instanceof BatchExecPythonCorrelate)) { ExecEdge newEdge = addExchangeAndReconnectEdge( - source.getInputEdges().get(0), inputProperty, strict); + tableConfig, + source.getInputEdges().get(0), + inputProperty, + strict, + true); source.setInputEdges(Collections.singletonList(newEdge)); } BatchExecExchange exchange = createExchangeWithKeepInputAsIsDistribution( - inputProperty, strict, (RowType) edge.getOutputType()); + tableConfig, inputProperty, strict, (RowType) edge.getOutputType()); ExecEdge newEdge = new ExecEdge(source, exchange, edge.getShuffle(), edge.getExchangeMode()); exchange.setInputEdges(Collections.singletonList(newEdge)); @@ -182,7 +200,10 @@ private ExecEdge addExchangeAndReconnectEdge( } private BatchExecExchange createExchangeWithKeepInputAsIsDistribution( - InputProperty inputProperty, boolean strict, RowType outputRowType) { + ReadableConfig tableConfig, + InputProperty inputProperty, + boolean strict, + RowType outputRowType) { InputProperty newInputProperty = InputProperty.builder() .requiredDistribution( @@ -191,7 +212,8 @@ private BatchExecExchange createExchangeWithKeepInputAsIsDistribution( .damBehavior(inputProperty.getDamBehavior()) .priority(inputProperty.getPriority()) .build(); - return new BatchExecExchange(newInputProperty, outputRowType, newInputProperty.toString()); + return new BatchExecExchange( + tableConfig, newInputProperty, outputRowType, newInputProperty.toString()); } private boolean hasExchangeInput(ExecEdge edge) { diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessor.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessor.java index 8cc3cc4909cd9..6495b2a2c906c 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessor.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessor.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.transformations.SourceTransformation; import org.apache.flink.streaming.api.transformations.StreamExchangeMode; import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge; @@ -79,7 +80,7 @@ public ExecNodeGraph process(ExecNodeGraph execGraph, ProcessorContext context) execGraph.getRootNodes(), InputProperty.DamBehavior.BLOCKING, StreamExchangeMode.PIPELINED, - context.getPlanner().getConfiguration()); + context.getPlanner().getTableConfig()); resolver.detectAndResolve(); } @@ -92,7 +93,8 @@ public ExecNodeGraph process(ExecNodeGraph execGraph, ProcessorContext context) optimizeMultipleInputGroups(orderedWrappers, context); // create the real multiple input nodes - List> newRootNodes = createMultipleInputNodes(rootWrappers); + List> newRootNodes = + createMultipleInputNodes(context.getPlanner().getTableConfig(), rootWrappers); return new ExecNodeGraph(newRootNodes); } @@ -451,23 +453,27 @@ static boolean isChainableSource(ExecNode node, ProcessorContext context) { // Multiple Input Nodes Creating // -------------------------------------------------------------------------------- - private List> createMultipleInputNodes(List rootWrappers) { + private List> createMultipleInputNodes( + ReadableConfig tableConfig, List rootWrappers) { List> result = new ArrayList<>(); Map> visitedMap = new HashMap<>(); for (ExecNodeWrapper rootWrapper : rootWrappers) { - result.add(getMultipleInputNode(rootWrapper, visitedMap)); + result.add(getMultipleInputNode(tableConfig, rootWrapper, visitedMap)); } return result; } private ExecNode getMultipleInputNode( - ExecNodeWrapper wrapper, Map> visitedMap) { + ReadableConfig tableConfig, + ExecNodeWrapper wrapper, + Map> visitedMap) { if (visitedMap.containsKey(wrapper)) { return visitedMap.get(wrapper); } for (int i = 0; i < wrapper.inputs.size(); i++) { - ExecNode multipleInputNode = getMultipleInputNode(wrapper.inputs.get(i), visitedMap); + ExecNode multipleInputNode = + getMultipleInputNode(tableConfig, wrapper.inputs.get(i), visitedMap); ExecEdge execEdge = ExecEdge.builder().source(multipleInputNode).target(wrapper.execNode).build(); wrapper.execNode.replaceInputEdge(i, execEdge); @@ -475,7 +481,7 @@ private ExecNode getMultipleInputNode( ExecNode ret; if (wrapper.group != null && wrapper == wrapper.group.root) { - ret = createMultipleInputNode(wrapper.group, visitedMap); + ret = createMultipleInputNode(tableConfig, wrapper.group, visitedMap); } else { ret = wrapper.execNode; } @@ -484,7 +490,9 @@ private ExecNode getMultipleInputNode( } private ExecNode createMultipleInputNode( - MultipleInputGroup group, Map> visitedMap) { + ReadableConfig tableConfig, + MultipleInputGroup group, + Map> visitedMap) { // calculate the inputs of the multiple input node List, InputProperty, ExecEdge>> inputs = new ArrayList<>(); for (ExecNodeWrapper member : group.members) { @@ -505,14 +513,16 @@ private ExecNode createMultipleInputNode( } if (isStreaming) { - return createStreamMultipleInputNode(group, inputs); + return createStreamMultipleInputNode(tableConfig, group, inputs); } else { - return createBatchMultipleInputNode(group, inputs); + return createBatchMultipleInputNode(tableConfig, group, inputs); } } private StreamExecMultipleInput createStreamMultipleInputNode( - MultipleInputGroup group, List, InputProperty, ExecEdge>> inputs) { + ReadableConfig tableConfig, + MultipleInputGroup group, + List, InputProperty, ExecEdge>> inputs) { ExecNode rootNode = group.root.execNode; List> inputNodes = new ArrayList<>(); for (Tuple3, InputProperty, ExecEdge> tuple3 : inputs) { @@ -523,6 +533,7 @@ private StreamExecMultipleInput createStreamMultipleInputNode( ExecNodeUtil.getMultipleInputDescription(rootNode, inputNodes, new ArrayList<>()); StreamExecMultipleInput multipleInput = new StreamExecMultipleInput( + tableConfig, inputNodes.stream() .map(i -> InputProperty.DEFAULT) .collect(Collectors.toList()), @@ -538,7 +549,9 @@ private StreamExecMultipleInput createStreamMultipleInputNode( } private BatchExecMultipleInput createBatchMultipleInputNode( - MultipleInputGroup group, List, InputProperty, ExecEdge>> inputs) { + ReadableConfig tableConfig, + MultipleInputGroup group, + List, InputProperty, ExecEdge>> inputs) { // first calculate the input orders using InputPriorityConflictResolver Set> inputSet = new HashSet<>(); for (Tuple3, InputProperty, ExecEdge> tuple3 : inputs) { @@ -571,7 +584,8 @@ private BatchExecMultipleInput createBatchMultipleInputNode( String description = ExecNodeUtil.getMultipleInputDescription(rootNode, inputNodes, inputProperties); BatchExecMultipleInput multipleInput = - new BatchExecMultipleInput(inputProperties, rootNode, originalEdges, description); + new BatchExecMultipleInput( + tableConfig, inputProperties, rootNode, originalEdges, description); List inputEdges = new ArrayList<>(inputNodes.size()); for (ExecNode inputNode : inputNodes) { diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityConflictResolver.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityConflictResolver.java index 655ca5bdc7a92..a6588df84d9d8 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityConflictResolver.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityConflictResolver.java @@ -43,7 +43,7 @@ public class InputPriorityConflictResolver extends InputPriorityGraphGenerator { private final StreamExchangeMode exchangeMode; - private final ReadableConfig configuration; + private final ReadableConfig tableConfig; /** * Create a {@link InputPriorityConflictResolver} for the given {@link ExecNode} graph. @@ -58,10 +58,10 @@ public InputPriorityConflictResolver( List> roots, InputProperty.DamBehavior safeDamBehavior, StreamExchangeMode exchangeMode, - ReadableConfig configuration) { + ReadableConfig tableConfig) { super(roots, Collections.emptySet(), safeDamBehavior); this.exchangeMode = exchangeMode; - this.configuration = configuration; + this.tableConfig = tableConfig; } public void detectAndResolve() { @@ -87,7 +87,10 @@ protected void resolveInputPriorityConflict(ExecNode node, int higherInput, i // we should split it into two nodes BatchExecExchange newExchange = new BatchExecExchange( - inputProperty, (RowType) exchange.getOutputType(), "Exchange"); + tableConfig, + inputProperty, + (RowType) exchange.getOutputType(), + "Exchange"); newExchange.setRequiredExchangeMode(exchangeMode); newExchange.setInputEdges(exchange.getInputEdges()); newNode = newExchange; @@ -95,6 +98,7 @@ protected void resolveInputPriorityConflict(ExecNode node, int higherInput, i // create new BatchExecExchange with new inputProperty BatchExecExchange newExchange = new BatchExecExchange( + tableConfig, inputProperty, (RowType) exchange.getOutputType(), exchange.getDescription()); @@ -138,7 +142,10 @@ private BatchExecExchange createExchange(ExecNode node, int idx) { .build(); BatchExecExchange exchange = new BatchExecExchange( - newInputProperty, (RowType) inputNode.getOutputType(), "Exchange"); + tableConfig, + newInputProperty, + (RowType) inputNode.getOutputType(), + "Exchange"); exchange.setRequiredExchangeMode(exchangeMode); ExecEdge execEdge = ExecEdge.builder().source(inputNode).target(exchange).build(); exchange.setInputEdges(Collections.singletonList(execEdge)); @@ -170,7 +177,7 @@ protected void visitNode(ExecNode node) { } private InputProperty.DamBehavior getDamBehavior() { - if (getBatchStreamExchangeMode(configuration, exchangeMode) == StreamExchangeMode.BATCH) { + if (getBatchStreamExchangeMode(tableConfig, exchangeMode) == StreamExchangeMode.BATCH) { return InputProperty.DamBehavior.BLOCKING; } else { return InputProperty.DamBehavior.PIPELINED; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonDeserializer.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonDeserializer.java new file mode 100644 index 0000000000000..d5cf4321e68e7 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonDeserializer.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.nodes.exec.serde; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.type.TypeReference; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.StdDeserializer; + +import java.io.IOException; +import java.util.Map; + +/** + * Custom deserializer for {@link Configuration} used for {@link ExecNodeBase#getPersistedConfig}. + */ +@Internal +class ConfigurationJsonDeserializer extends StdDeserializer { + + private static final TypeReference MAP_STRING_STRING_TYPE_REF = + new TypeReference>() {}; + + ConfigurationJsonDeserializer() { + super(Configuration.class); + } + + @Override + public Configuration deserialize( + JsonParser jsonParser, DeserializationContext deserializationContext) + throws IOException { + return Configuration.fromMap(jsonParser.readValueAs(MAP_STRING_STRING_TYPE_REF)); + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonSerializer.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonSerializer.java new file mode 100644 index 0000000000000..0a1c0afba3168 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonSerializer.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.nodes.exec.serde; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.SerializerProvider; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ser.std.StdSerializer; + +import java.io.IOException; +import java.util.TreeMap; + +/** + * Custom serializer for {@link Configuration} used for {@link ExecNodeBase#getPersistedConfig()}. + */ +@Internal +class ConfigurationJsonSerializer extends StdSerializer { + + ConfigurationJsonSerializer() { + super(Configuration.class); + } + + @Override + public void serialize( + Configuration configuration, + JsonGenerator jsonGenerator, + SerializerProvider serializerProvider) + throws IOException { + // Use TreeMap to have a nice -sorted by key- output of the config options. + serializerProvider.defaultSerializeValue( + new TreeMap<>(configuration.toMap()), jsonGenerator); + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonSerializerFilter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonSerializerFilter.java new file mode 100644 index 0000000000000..1d1a4a5e82d0c --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonSerializerFilter.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.nodes.exec.serde; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase; + +/** + * Custom filtering for {@link Configuration} used by {@link ExecNodeBase#getPersistedConfig()} to + * avoid serializing null or empty configurations. + */ +@Internal +public class ConfigurationJsonSerializerFilter { + + @Override + public boolean equals(Object obj) { + if (obj == null) { + return true; + } + if (obj instanceof Configuration) { + Configuration other = (Configuration) obj; + return other.toMap().isEmpty(); + } + return true; + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/JsonSerdeUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/JsonSerdeUtil.java index f044d0d33cd0a..8a601601d3f99 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/JsonSerdeUtil.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/JsonSerdeUtil.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ContextResolvedTable; @@ -139,6 +140,7 @@ private static Module createFlinkTableJacksonModule() { private static void registerSerializers(SimpleModule module) { module.addSerializer(new ExecNodeGraphJsonSerializer()); module.addSerializer(new FlinkVersionJsonSerializer()); + module.addSerializer(new ConfigurationJsonSerializer()); module.addSerializer(new ObjectIdentifierJsonSerializer()); module.addSerializer(new LogicalTypeJsonSerializer()); module.addSerializer(new DataTypeJsonSerializer()); @@ -161,6 +163,7 @@ private static void registerSerializers(SimpleModule module) { private static void registerDeserializers(SimpleModule module) { module.addDeserializer(ExecNodeGraph.class, new ExecNodeGraphJsonDeserializer()); module.addDeserializer(FlinkVersion.class, new FlinkVersionJsonDeserializer()); + module.addDeserializer(ReadableConfig.class, new ConfigurationJsonDeserializer()); module.addDeserializer(ObjectIdentifier.class, new ObjectIdentifierJsonDeserializer()); module.addDeserializer(LogicalType.class, new LogicalTypeJsonDeserializer()); module.addDeserializer(RowType.class, (StdDeserializer) new LogicalTypeJsonDeserializer()); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonDeserializer.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonDeserializer.java index 8e52495ea3fb3..cbb7df0568031 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonDeserializer.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonDeserializer.java @@ -46,8 +46,10 @@ import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlOperator; import org.apache.calcite.sql.SqlSyntax; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.sql.validate.SqlNameMatchers; @@ -91,6 +93,7 @@ import static org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonSerializer.FIELD_NAME_OPERANDS; import static org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonSerializer.FIELD_NAME_RANGES; import static org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonSerializer.FIELD_NAME_SARG; +import static org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonSerializer.FIELD_NAME_SQL_KIND; import static org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonSerializer.FIELD_NAME_SYMBOL; import static org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonSerializer.FIELD_NAME_SYNTAX; import static org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonSerializer.FIELD_NAME_SYSTEM_NAME; @@ -340,6 +343,9 @@ static SqlOperator deserializeSqlOperator(JsonNode jsonNode, SerdeContext serdeC } else if (jsonNode.has(FIELD_NAME_SYSTEM_NAME)) { return deserializeSystemFunction( jsonNode.required(FIELD_NAME_SYSTEM_NAME).asText(), syntax, serdeContext); + } else if (jsonNode.has(FIELD_NAME_SQL_KIND)) { + return deserializeInternalFunction( + syntax, SqlKind.valueOf(jsonNode.get(FIELD_NAME_SQL_KIND).asText())); } else { throw new TableException("Invalid function call."); } @@ -375,6 +381,13 @@ private static SqlOperator deserializeInternalFunction( if (latestOperator.isPresent()) { return latestOperator.get(); } + + Optional sqlStdOperator = + lookupOptionalSqlStdOperator(publicName, syntax, null); + if (sqlStdOperator.isPresent()) { + return sqlStdOperator.get(); + } + throw new TableException( String.format( "Could not resolve internal system function '%s'. " @@ -382,6 +395,19 @@ private static SqlOperator deserializeInternalFunction( internalName)); } + private static SqlOperator deserializeInternalFunction(SqlSyntax syntax, SqlKind sqlKind) { + final Optional stdOperator = lookupOptionalSqlStdOperator("", syntax, sqlKind); + if (stdOperator.isPresent()) { + return stdOperator.get(); + } + + throw new TableException( + String.format( + "Could not resolve internal system function '%s'. " + + "This is a bug, please file an issue.", + sqlKind.name())); + } + private static SqlOperator deserializeFunctionClass( JsonNode jsonNode, SerdeContext serdeContext) { final String className = jsonNode.required(FIELD_NAME_CLASS).asText(); @@ -506,6 +532,26 @@ private static Optional lookupOptionalSqlOperator( } } + private static Optional lookupOptionalSqlStdOperator( + String operatorName, SqlSyntax syntax, @Nullable SqlKind sqlKind) { + List foundOperators = new ArrayList<>(); + // try to find operator from std operator table. + SqlStdOperatorTable.instance() + .lookupOperatorOverloads( + new SqlIdentifier(operatorName, new SqlParserPos(0, 0)), + null, // category + syntax, + foundOperators, + SqlNameMatchers.liberal()); + if (foundOperators.size() == 1) { + return Optional.of(foundOperators.get(0)); + } + // in case different operator has the same kind, check with both name and kind. + return foundOperators.stream() + .filter(o -> sqlKind != null && o.getKind() == sqlKind) + .findFirst(); + } + private static TableException missingSystemFunction(String systemName) { return new TableException( String.format( diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonSerializer.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonSerializer.java index 7e38317e729d3..5e788723cd3ca 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonSerializer.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonSerializer.java @@ -116,6 +116,7 @@ final class RexNodeJsonSerializer extends StdSerializer { static final String FIELD_NAME_SYSTEM_NAME = "systemName"; static final String FIELD_NAME_CATALOG_NAME = "catalogName"; static final String FIELD_NAME_SYNTAX = "syntax"; + static final String FIELD_NAME_SQL_KIND = "sqlKind"; static final String FIELD_NAME_CLASS = "class"; RexNodeJsonSerializer() { @@ -393,10 +394,15 @@ static void serializeSqlOperator( || operator instanceof AggSqlFunction) { throw legacyException(operator.toString()); } else { - // We assume that all regular SqlOperators are internal. Only the function definitions - // stack is exposed to the user and can thus be external. - gen.writeStringField( - FIELD_NAME_INTERNAL_NAME, BuiltInSqlOperator.toQualifiedName(operator)); + if (operator.getName().isEmpty()) { + gen.writeStringField(FIELD_NAME_SQL_KIND, operator.getKind().name()); + } else { + // We assume that all regular SqlOperators are internal. Only the function + // definitions + // stack is exposed to the user and can thus be external. + gen.writeStringField( + FIELD_NAME_INTERNAL_NAME, BuiltInSqlOperator.toQualifiedName(operator)); + } } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/SerdeContext.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/SerdeContext.java index 12741328a7c77..f02afde1fab0e 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/SerdeContext.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/SerdeContext.java @@ -19,7 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.serde; import org.apache.flink.annotation.Internal; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.delegation.Parser; import org.apache.flink.table.planner.calcite.FlinkContext; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; @@ -71,8 +71,8 @@ public Parser getParser() { return parser; } - public Configuration getConfiguration() { - return flinkContext.getTableConfig().getConfiguration(); + public ReadableConfig getConfiguration() { + return flinkContext.getTableConfig(); } public ClassLoader getClassLoader() { diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSinkSpec.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSinkSpec.java index 73cb36c2d4cd6..4f092a24ba3f3 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSinkSpec.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSinkSpec.java @@ -87,7 +87,7 @@ public DynamicTableSink getTableSink(FlinkContext flinkContext) { contextResolvedTable.getIdentifier(), contextResolvedTable.getResolvedTable(), loadOptionsFromCatalogTable(contextResolvedTable, flinkContext), - flinkContext.getTableConfig().getConfiguration(), + flinkContext.getTableConfig(), flinkContext.getClassLoader(), contextResolvedTable.isTemporary()); if (sinkAbilities != null) { diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSourceSpec.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSourceSpec.java index b0fd9ce71aabd..3cabdb9c89f5b 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSourceSpec.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSourceSpec.java @@ -80,7 +80,7 @@ private DynamicTableSource getTableSource(FlinkContext flinkContext) { contextResolvedTable.getIdentifier(), contextResolvedTable.getResolvedTable(), loadOptionsFromCatalogTable(contextResolvedTable, flinkContext), - flinkContext.getTableConfig().getConfiguration(), + flinkContext.getTableConfig(), flinkContext.getClassLoader(), contextResolvedTable.isTemporary()); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSpecBase.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSpecBase.java index b8d941f976992..3521dbfa3ac45 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSpecBase.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSpecBase.java @@ -38,10 +38,7 @@ Map loadOptionsFromCatalogTable( // In case of CatalogPlanRestore.IDENTIFIER, getCatalogTable() already returns the table // loaded from the catalog final TableConfigOptions.CatalogPlanRestore catalogPlanRestore = - flinkContext - .getTableConfig() - .getConfiguration() - .get(TableConfigOptions.PLAN_RESTORE_CATALOG_OBJECTS); + flinkContext.getTableConfig().get(TableConfigOptions.PLAN_RESTORE_CATALOG_OBJECTS); if (!contextResolvedTable.isPermanent() || catalogPlanRestore != TableConfigOptions.CatalogPlanRestore.ALL) { return Collections.emptyMap(); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecAggregateBase.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecAggregateBase.java index fde9fda4bc4c6..ee33e0a580c34 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecAggregateBase.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecAggregateBase.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase; @@ -43,10 +44,11 @@ public abstract class StreamExecAggregateBase extends ExecNodeBase protected StreamExecAggregateBase( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List inputProperties, LogicalType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCalc.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCalc.java index 90112df84c2fe..72b4be1a87160 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCalc.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCalc.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.FlinkVersion; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -48,6 +49,7 @@ public class StreamExecCalc extends CommonExecCalc implements StreamExecNode { public StreamExecCalc( + ReadableConfig tableConfig, List projection, @Nullable RexNode condition, InputProperty inputProperty, @@ -56,6 +58,7 @@ public StreamExecCalc( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecCalc.class), + ExecNodeContext.newPersistedConfig(StreamExecCalc.class, tableConfig), projection, condition, Collections.singletonList(inputProperty), @@ -67,6 +70,7 @@ public StreamExecCalc( public StreamExecCalc( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_PROJECTION) List projection, @JsonProperty(FIELD_NAME_CONDITION) @Nullable RexNode condition, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @@ -75,6 +79,7 @@ public StreamExecCalc( super( id, context, + persistedConfig, projection, condition, TableStreamOperator.class, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecChangelogNormalize.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecChangelogNormalize.java index c3d0677ed4f12..077c950847687 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecChangelogNormalize.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecChangelogNormalize.java @@ -21,6 +21,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.KeyedProcessOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -64,7 +65,6 @@ name = "stream-exec-changelog-normalize", version = 1, consumedOptions = { - "table.exec.state.ttl", "table.exec.mini-batch.enabled", "table.exec.mini-batch.size", }, @@ -86,6 +86,7 @@ public class StreamExecChangelogNormalize extends ExecNodeBase private final boolean generateUpdateBefore; public StreamExecChangelogNormalize( + ReadableConfig tableConfig, int[] uniqueKeys, boolean generateUpdateBefore, InputProperty inputProperty, @@ -94,6 +95,7 @@ public StreamExecChangelogNormalize( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecChangelogNormalize.class), + ExecNodeContext.newPersistedConfig(StreamExecChangelogNormalize.class, tableConfig), uniqueKeys, generateUpdateBefore, Collections.singletonList(inputProperty), @@ -105,12 +107,13 @@ public StreamExecChangelogNormalize( public StreamExecChangelogNormalize( @JsonProperty(FIELD_NAME_ID) Integer id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_UNIQUE_KEYS) int[] uniqueKeys, @JsonProperty(FIELD_NAME_GENERATE_UPDATE_BEFORE) boolean generateUpdateBefore, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); this.uniqueKeys = uniqueKeys; this.generateUpdateBefore = generateUpdateBefore; } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCorrelate.java index c843ecfbe8afc..eb493898fab4b 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCorrelate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCorrelate.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.FlinkVersion; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -52,6 +53,7 @@ public class StreamExecCorrelate extends CommonExecCorrelate implements StreamExecNode { public StreamExecCorrelate( + ReadableConfig tableConfig, FlinkJoinType joinType, RexCall invocation, @Nullable RexNode condition, @@ -61,6 +63,7 @@ public StreamExecCorrelate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecCorrelate.class), + ExecNodeContext.newPersistedConfig(StreamExecCorrelate.class, tableConfig), joinType, invocation, condition, @@ -73,6 +76,7 @@ public StreamExecCorrelate( public StreamExecCorrelate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_JOIN_TYPE) FlinkJoinType joinType, @JsonProperty(FIELD_NAME_FUNCTION_CALL) RexNode invocation, @JsonProperty(FIELD_NAME_CONDITION) @Nullable RexNode condition, @@ -82,6 +86,7 @@ public StreamExecCorrelate( super( id, context, + persistedConfig, joinType, (RexCall) invocation, condition, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDataStreamScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDataStreamScan.java index 7ea8db3fafbf1..4b599284cf2e8 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDataStreamScan.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDataStreamScan.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.calcite.FlinkRelBuilder; @@ -63,6 +64,7 @@ public class StreamExecDataStreamScan extends ExecNodeBase private final List qualifiedName; public StreamExecDataStreamScan( + ReadableConfig tableConfig, DataStream dataStream, DataType sourceType, int[] fieldIndexes, @@ -73,6 +75,7 @@ public StreamExecDataStreamScan( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecDataStreamScan.class), + ExecNodeContext.newPersistedConfig(StreamExecDataStreamScan.class, tableConfig), Collections.emptyList(), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeduplicate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeduplicate.java index c4888835661ec..93425a9e35e0e 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeduplicate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeduplicate.java @@ -74,7 +74,6 @@ name = "stream-exec-deduplicate", version = 1, consumedOptions = { - "table.exec.state.ttl", "table.exec.mini-batch.enabled", "table.exec.mini-batch.size", "table.exec.deduplicate.insert-update-after-sensitive-enabled", @@ -106,6 +105,7 @@ public class StreamExecDeduplicate extends ExecNodeBase private final boolean generateUpdateBefore; public StreamExecDeduplicate( + ReadableConfig tableConfig, int[] uniqueKeys, boolean isRowtime, boolean keepLastRow, @@ -116,6 +116,7 @@ public StreamExecDeduplicate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecDeduplicate.class), + ExecNodeContext.newPersistedConfig(StreamExecDeduplicate.class, tableConfig), uniqueKeys, isRowtime, keepLastRow, @@ -129,6 +130,7 @@ public StreamExecDeduplicate( public StreamExecDeduplicate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_UNIQUE_KEYS) int[] uniqueKeys, @JsonProperty(FIELD_NAME_IS_ROWTIME) boolean isRowtime, @JsonProperty(FIELD_NAME_KEEP_LAST_ROW) boolean keepLastRow, @@ -136,7 +138,7 @@ public StreamExecDeduplicate( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.uniqueKeys = checkNotNull(uniqueKeys); this.isRowtime = isRowtime; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDropUpdateBefore.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDropUpdateBefore.java index eb8cdcb54a199..cc2ef75f809fe 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDropUpdateBefore.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDropUpdateBefore.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.StreamFilter; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; @@ -57,10 +58,14 @@ public class StreamExecDropUpdateBefore extends ExecNodeBase public static final String DROP_UPDATE_BEFORE_TRANSFORMATION = "drop-update-before"; public StreamExecDropUpdateBefore( - InputProperty inputProperty, RowType outputType, String description) { + ReadableConfig tableConfig, + InputProperty inputProperty, + RowType outputType, + String description) { this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecDropUpdateBefore.class), + ExecNodeContext.newPersistedConfig(StreamExecDropUpdateBefore.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); @@ -70,10 +75,11 @@ public StreamExecDropUpdateBefore( public StreamExecDropUpdateBefore( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); } @SuppressWarnings("unchecked") diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecExchange.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecExchange.java index f7b5d524aad98..c8d37f23c7162 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecExchange.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecExchange.java @@ -21,6 +21,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.transformations.PartitionTransformation; import org.apache.flink.streaming.runtime.partitioner.GlobalPartitioner; import org.apache.flink.streaming.runtime.partitioner.KeyGroupStreamPartitioner; @@ -64,10 +65,15 @@ public class StreamExecExchange extends CommonExecExchange implements StreamExec public static final String EXCHANGE_TRANSFORMATION = "exchange"; - public StreamExecExchange(InputProperty inputProperty, RowType outputType, String description) { + public StreamExecExchange( + ReadableConfig tableConfig, + InputProperty inputProperty, + RowType outputType, + String description) { this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecExchange.class), + ExecNodeContext.newPersistedConfig(StreamExecExchange.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); @@ -77,10 +83,11 @@ public StreamExecExchange(InputProperty inputProperty, RowType outputType, Strin public StreamExecExchange( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecExpand.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecExpand.java index d46c30f87f2ce..e1ec85678fc0d 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecExpand.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecExpand.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.FlinkVersion; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -45,6 +46,7 @@ public class StreamExecExpand extends CommonExecExpand implements StreamExecNode { public StreamExecExpand( + ReadableConfig tableConfig, List> projects, InputProperty inputProperty, RowType outputType, @@ -52,6 +54,7 @@ public StreamExecExpand( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecExpand.class), + ExecNodeContext.newPersistedConfig(StreamExecExpand.class, tableConfig), projects, Collections.singletonList(inputProperty), outputType, @@ -62,6 +65,7 @@ public StreamExecExpand( public StreamExecExpand( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_PROJECTS) List> projects, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @@ -69,6 +73,7 @@ public StreamExecExpand( super( id, context, + persistedConfig, projects, true, // retainHeader inputProperties, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalGroupAggregate.java index 9e68aa53864dd..08dd8c146eeb1 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalGroupAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalGroupAggregate.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.table.api.TableException; @@ -74,11 +75,7 @@ @ExecNodeMetadata( name = "stream-exec-global-group-aggregate", version = 1, - consumedOptions = { - "table.exec.state.ttl", - "table.exec.mini-batch.enabled", - "table.exec.mini-batch.size" - }, + consumedOptions = {"table.exec.mini-batch.enabled", "table.exec.mini-batch.size"}, producedTransformations = StreamExecGlobalGroupAggregate.GLOBAL_GROUP_AGGREGATE_TRANSFORMATION, minPlanVersion = FlinkVersion.v1_15, @@ -120,6 +117,7 @@ public class StreamExecGlobalGroupAggregate extends StreamExecAggregateBase { protected final Integer indexOfCountStar; public StreamExecGlobalGroupAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, boolean[] aggCallNeedRetractions, @@ -133,6 +131,8 @@ public StreamExecGlobalGroupAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecGlobalGroupAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecGlobalGroupAggregate.class, tableConfig), grouping, aggCalls, aggCallNeedRetractions, @@ -149,6 +149,7 @@ public StreamExecGlobalGroupAggregate( public StreamExecGlobalGroupAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_GROUPING) int[] grouping, @JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls, @JsonProperty(FIELD_NAME_AGG_CALL_NEED_RETRACTIONS) boolean[] aggCallNeedRetractions, @@ -159,7 +160,7 @@ public StreamExecGlobalGroupAggregate( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); this.grouping = checkNotNull(grouping); this.aggCalls = checkNotNull(aggCalls); this.aggCallNeedRetractions = checkNotNull(aggCallNeedRetractions); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalWindowAggregate.java index d3c2710bf7dde..41c6a9c1b9dac 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalWindowAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalWindowAggregate.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -39,6 +40,7 @@ import org.apache.flink.table.planner.plan.utils.AggregateUtil; import org.apache.flink.table.planner.plan.utils.KeySelectorUtil; import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; +import org.apache.flink.table.planner.utils.TableConfigUtils; import org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction; import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty; import org.apache.flink.table.runtime.groupwindow.WindowProperty; @@ -98,6 +100,7 @@ public class StreamExecGlobalWindowAggregate extends StreamExecWindowAggregateBa private final RowType localAggInputRowType; public StreamExecGlobalWindowAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, WindowingStrategy windowing, @@ -109,6 +112,8 @@ public StreamExecGlobalWindowAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecGlobalWindowAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecGlobalWindowAggregate.class, tableConfig), grouping, aggCalls, windowing, @@ -123,6 +128,7 @@ public StreamExecGlobalWindowAggregate( public StreamExecGlobalWindowAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_GROUPING) int[] grouping, @JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls, @JsonProperty(FIELD_NAME_WINDOWING) WindowingStrategy windowing, @@ -132,7 +138,7 @@ public StreamExecGlobalWindowAggregate( @JsonProperty(FIELD_NAME_LOCAL_AGG_INPUT_ROW_TYPE) RowType localAggInputRowType, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); this.grouping = checkNotNull(grouping); this.aggCalls = checkNotNull(aggCalls); this.windowing = checkNotNull(windowing); @@ -151,7 +157,8 @@ protected Transformation translateToPlanInternal( final ZoneId shiftTimeZone = TimeWindowUtil.getShiftTimeZone( - windowing.getTimeAttributeType(), config.getLocalTimeZone()); + windowing.getTimeAttributeType(), + TableConfigUtils.getLocalTimeZone(config)); final SliceAssigner sliceAssigner = createSliceAssigner(windowing, shiftTimeZone); final AggregateInfoList localAggInfoList = diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupAggregate.java index c6d11e6090dcd..2af340bda8df7 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupAggregate.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.KeyedProcessOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -73,11 +74,7 @@ @ExecNodeMetadata( name = "stream-exec-group-aggregate", version = 1, - consumedOptions = { - "table.exec.state.ttl", - "table.exec.mini-batch.enabled", - "table.exec.mini-batch.size" - }, + consumedOptions = {"table.exec.mini-batch.enabled", "table.exec.mini-batch.size"}, producedTransformations = StreamExecGroupAggregate.GROUP_AGGREGATE_TRANSFORMATION, minPlanVersion = FlinkVersion.v1_15, minStateVersion = FlinkVersion.v1_15) @@ -106,6 +103,7 @@ public class StreamExecGroupAggregate extends StreamExecAggregateBase { private final boolean needRetraction; public StreamExecGroupAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, boolean[] aggCallNeedRetractions, @@ -117,6 +115,7 @@ public StreamExecGroupAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecGroupAggregate.class), + ExecNodeContext.newPersistedConfig(StreamExecGroupAggregate.class, tableConfig), grouping, aggCalls, aggCallNeedRetractions, @@ -131,6 +130,7 @@ public StreamExecGroupAggregate( public StreamExecGroupAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_GROUPING) int[] grouping, @JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls, @JsonProperty(FIELD_NAME_AGG_CALL_NEED_RETRACTIONS) boolean[] aggCallNeedRetractions, @@ -139,7 +139,7 @@ public StreamExecGroupAggregate( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); this.grouping = checkNotNull(grouping); this.aggCalls = checkNotNull(aggCalls); this.aggCallNeedRetractions = checkNotNull(aggCallNeedRetractions); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupTableAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupTableAggregate.java index cd759445a8a0b..d242deec9c4d5 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupTableAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupTableAggregate.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.KeyedProcessOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -72,6 +73,7 @@ public class StreamExecGroupTableAggregate extends ExecNodeBase private final boolean needRetraction; public StreamExecGroupTableAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, boolean[] aggCallNeedRetractions, @@ -83,6 +85,8 @@ public StreamExecGroupTableAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecGroupTableAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecGroupTableAggregate.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupWindowAggregate.java index 0e330975ace06..5c96b53896d25 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupWindowAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupWindowAggregate.java @@ -46,6 +46,7 @@ import org.apache.flink.table.planner.plan.utils.KeySelectorUtil; import org.apache.flink.table.planner.plan.utils.WindowEmitStrategy; import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; +import org.apache.flink.table.planner.utils.TableConfigUtils; import org.apache.flink.table.runtime.generated.GeneratedClass; import org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction; import org.apache.flink.table.runtime.generated.GeneratedNamespaceTableAggsHandleFunction; @@ -104,7 +105,6 @@ version = 1, consumedOptions = { "table.local-time-zone", - "table.exec.state.ttl", "table.exec.mini-batch.enabled", "table.exec.mini-batch.size" }, @@ -138,6 +138,7 @@ public class StreamExecGroupWindowAggregate extends StreamExecAggregateBase { private final boolean needRetraction; public StreamExecGroupWindowAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, LogicalWindow window, @@ -149,6 +150,8 @@ public StreamExecGroupWindowAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecGroupWindowAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecGroupWindowAggregate.class, tableConfig), grouping, aggCalls, window, @@ -163,6 +166,7 @@ public StreamExecGroupWindowAggregate( public StreamExecGroupWindowAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_GROUPING) int[] grouping, @JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls, @JsonProperty(FIELD_NAME_WINDOW) LogicalWindow window, @@ -172,7 +176,7 @@ public StreamExecGroupWindowAggregate( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.grouping = checkNotNull(grouping); this.aggCalls = checkNotNull(aggCalls); @@ -226,7 +230,7 @@ protected Transformation translateToPlanInternal( final ZoneId shiftTimeZone = TimeWindowUtil.getShiftTimeZone( window.timeAttribute().getOutputDataType().getLogicalType(), - config.getLocalTimeZone()); + TableConfigUtils.getLocalTimeZone(config)); final boolean[] aggCallNeedRetractions = new boolean[aggCalls.length]; Arrays.fill(aggCallNeedRetractions, needRetraction); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIncrementalGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIncrementalGroupAggregate.java index 93228992b24a4..3a419e09638df 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIncrementalGroupAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIncrementalGroupAggregate.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.table.data.RowData; @@ -62,11 +63,7 @@ @ExecNodeMetadata( name = "stream-exec-incremental-group-aggregate", version = 1, - consumedOptions = { - "table.exec.state.ttl", - "table.exec.mini-batch.enabled", - "table.exec.mini-batch.size" - }, + consumedOptions = {"table.exec.mini-batch.enabled", "table.exec.mini-batch.size"}, producedTransformations = StreamExecIncrementalGroupAggregate.INCREMENTAL_GROUP_AGGREGATE_TRANSFORMATION, minPlanVersion = FlinkVersion.v1_15, @@ -110,6 +107,7 @@ public class StreamExecIncrementalGroupAggregate extends StreamExecAggregateBase private final boolean partialAggNeedRetraction; public StreamExecIncrementalGroupAggregate( + ReadableConfig tableConfig, int[] partialAggGrouping, int[] finalAggGrouping, AggregateCall[] partialOriginalAggCalls, @@ -122,6 +120,8 @@ public StreamExecIncrementalGroupAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecIncrementalGroupAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecIncrementalGroupAggregate.class, tableConfig), partialAggGrouping, finalAggGrouping, partialOriginalAggCalls, @@ -137,6 +137,7 @@ public StreamExecIncrementalGroupAggregate( public StreamExecIncrementalGroupAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_PARTIAL_AGG_GROUPING) int[] partialAggGrouping, @JsonProperty(FIELD_NAME_FINAL_AGG_GROUPING) int[] finalAggGrouping, @JsonProperty(FIELD_NAME_PARTIAL_ORIGINAL_AGG_CALLS) @@ -148,7 +149,7 @@ public StreamExecIncrementalGroupAggregate( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); this.partialAggGrouping = checkNotNull(partialAggGrouping); this.finalAggGrouping = checkNotNull(finalAggGrouping); this.partialOriginalAggCalls = checkNotNull(partialOriginalAggCalls); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIntervalJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIntervalJoin.java index f9794b18382c4..4c6cf12213831 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIntervalJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIntervalJoin.java @@ -96,6 +96,7 @@ public class StreamExecIntervalJoin extends ExecNodeBase private final IntervalJoinSpec intervalJoinSpec; public StreamExecIntervalJoin( + ReadableConfig tableConfig, IntervalJoinSpec intervalJoinSpec, InputProperty leftInputProperty, InputProperty rightInputProperty, @@ -104,6 +105,7 @@ public StreamExecIntervalJoin( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecIntervalJoin.class), + ExecNodeContext.newPersistedConfig(StreamExecIntervalJoin.class, tableConfig), intervalJoinSpec, Lists.newArrayList(leftInputProperty, rightInputProperty), outputType, @@ -114,11 +116,12 @@ public StreamExecIntervalJoin( public StreamExecIntervalJoin( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_INTERVAL_JOIN_SPEC) IntervalJoinSpec intervalJoinSpec, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); Preconditions.checkArgument(inputProperties.size() == 2); this.intervalJoinSpec = Preconditions.checkNotNull(intervalJoinSpec); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java index 5fb1624498728..e14ef9a11b576 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java @@ -21,6 +21,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.transformations.TwoInputTransformation; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; @@ -85,6 +86,7 @@ public class StreamExecJoin extends ExecNodeBase private final List rightUniqueKeys; public StreamExecJoin( + ReadableConfig tableConfig, JoinSpec joinSpec, List leftUniqueKeys, List rightUniqueKeys, @@ -95,6 +97,7 @@ public StreamExecJoin( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecJoin.class), + ExecNodeContext.newPersistedConfig(StreamExecJoin.class, tableConfig), joinSpec, leftUniqueKeys, rightUniqueKeys, @@ -107,13 +110,14 @@ public StreamExecJoin( public StreamExecJoin( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_JOIN_SPEC) JoinSpec joinSpec, @JsonProperty(FIELD_NAME_LEFT_UNIQUE_KEYS) List leftUniqueKeys, @JsonProperty(FIELD_NAME_RIGHT_UNIQUE_KEYS) List rightUniqueKeys, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 2); this.joinSpec = checkNotNull(joinSpec); this.leftUniqueKeys = leftUniqueKeys; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacySink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacySink.java index eeb68403c59b2..04fdde6c11b83 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacySink.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacySink.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -44,6 +45,7 @@ public class StreamExecLegacySink extends CommonExecLegacySink implements StreamExecNode { public StreamExecLegacySink( + ReadableConfig tableConfig, TableSink tableSink, @Nullable String[] upsertKeys, boolean needRetraction, @@ -53,6 +55,7 @@ public StreamExecLegacySink( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecLegacySink.class), + ExecNodeContext.newPersistedConfig(StreamExecLegacySink.class, tableConfig), tableSink, upsertKeys, needRetraction, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacyTableSourceScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacyTableSourceScan.java index c5b2128e32a0e..1bf47df754a6e 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacyTableSourceScan.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacyTableSourceScan.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.io.InputSplit; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -63,6 +64,7 @@ public class StreamExecLegacyTableSourceScan extends CommonExecLegacyTableSource implements StreamExecNode { public StreamExecLegacyTableSourceScan( + ReadableConfig tableConfig, TableSource tableSource, List qualifiedName, RowType outputType, @@ -70,6 +72,8 @@ public StreamExecLegacyTableSourceScan( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecLegacyTableSourceScan.class), + ExecNodeContext.newPersistedConfig( + StreamExecLegacyTableSourceScan.class, tableConfig), tableSource, qualifiedName, outputType, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLimit.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLimit.java index 8f255f9836b9f..2f673a04aedd7 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLimit.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLimit.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; @@ -45,7 +46,7 @@ @ExecNodeMetadata( name = "stream-exec-limit", version = 1, - consumedOptions = {"table.exec.rank.topn-cache-size", "table.exec.state.ttl"}, + consumedOptions = {"table.exec.rank.topn-cache-size"}, producedTransformations = StreamExecRank.RANK_TRANSFORMATION, minPlanVersion = FlinkVersion.v1_15, minStateVersion = FlinkVersion.v1_15) @@ -54,6 +55,7 @@ public class StreamExecLimit extends StreamExecRank { private final long limitEnd; public StreamExecLimit( + ReadableConfig tableConfig, long limitStart, long limitEnd, boolean generateUpdateBefore, @@ -64,6 +66,7 @@ public StreamExecLimit( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecLimit.class), + ExecNodeContext.newPersistedConfig(StreamExecLimit.class, tableConfig), new ConstantRankRange(limitStart + 1, limitEnd), getRankStrategy(needRetraction), generateUpdateBefore, @@ -76,6 +79,7 @@ public StreamExecLimit( public StreamExecLimit( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_RANK_RANG) ConstantRankRange rankRange, @JsonProperty(FIELD_NAME_RANK_STRATEGY) RankProcessStrategy rankStrategy, @JsonProperty(FIELD_NAME_GENERATE_UPDATE_BEFORE) boolean generateUpdateBefore, @@ -85,6 +89,7 @@ public StreamExecLimit( super( id, context, + persistedConfig, RankType.ROW_NUMBER, PartitionSpec.ALL_IN_ONE, SortSpec.ANY, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalGroupAggregate.java index 80a257a9c24e8..aefdcb974ff5e 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalGroupAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalGroupAggregate.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.CodeGeneratorContext; import org.apache.flink.table.planner.codegen.agg.AggsHandlerCodeGenerator; @@ -85,6 +86,7 @@ public class StreamExecLocalGroupAggregate extends StreamExecAggregateBase { private final boolean needRetraction; public StreamExecLocalGroupAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, boolean[] aggCallNeedRetractions, @@ -95,6 +97,8 @@ public StreamExecLocalGroupAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecLocalGroupAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecLocalGroupAggregate.class, tableConfig), grouping, aggCalls, aggCallNeedRetractions, @@ -108,6 +112,7 @@ public StreamExecLocalGroupAggregate( public StreamExecLocalGroupAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_GROUPING) int[] grouping, @JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls, @JsonProperty(FIELD_NAME_AGG_CALL_NEED_RETRACTIONS) boolean[] aggCallNeedRetractions, @@ -115,7 +120,7 @@ public StreamExecLocalGroupAggregate( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); this.grouping = checkNotNull(grouping); this.aggCalls = checkNotNull(aggCalls); this.aggCallNeedRetractions = checkNotNull(aggCallNeedRetractions); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalWindowAggregate.java index a701962f99042..ae138297449ac 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalWindowAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalWindowAggregate.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.table.data.RowData; @@ -38,6 +39,7 @@ import org.apache.flink.table.planner.plan.utils.AggregateUtil; import org.apache.flink.table.planner.plan.utils.KeySelectorUtil; import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; +import org.apache.flink.table.planner.utils.TableConfigUtils; import org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction; import org.apache.flink.table.runtime.keyselector.RowDataKeySelector; import org.apache.flink.table.runtime.operators.aggregate.window.LocalSlicingWindowAggOperator; @@ -93,6 +95,7 @@ public class StreamExecLocalWindowAggregate extends StreamExecWindowAggregateBas private final WindowingStrategy windowing; public StreamExecLocalWindowAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, WindowingStrategy windowing, @@ -102,6 +105,8 @@ public StreamExecLocalWindowAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecLocalWindowAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecLocalWindowAggregate.class, tableConfig), grouping, aggCalls, windowing, @@ -114,13 +119,14 @@ public StreamExecLocalWindowAggregate( public StreamExecLocalWindowAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_GROUPING) int[] grouping, @JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls, @JsonProperty(FIELD_NAME_WINDOWING) WindowingStrategy windowing, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); this.grouping = checkNotNull(grouping); this.aggCalls = checkNotNull(aggCalls); this.windowing = checkNotNull(windowing); @@ -137,7 +143,8 @@ protected Transformation translateToPlanInternal( final ZoneId shiftTimeZone = TimeWindowUtil.getShiftTimeZone( - windowing.getTimeAttributeType(), config.getLocalTimeZone()); + windowing.getTimeAttributeType(), + TableConfigUtils.getLocalTimeZone(config)); final SliceAssigner sliceAssigner = createSliceAssigner(windowing, shiftTimeZone); final AggregateInfoList aggInfoList = diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLookupJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLookupJoin.java index fdc80776201ca..8954958882726 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLookupJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLookupJoin.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.FlinkVersion; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata; @@ -49,6 +50,7 @@ minStateVersion = FlinkVersion.v1_15) public class StreamExecLookupJoin extends CommonExecLookupJoin implements StreamExecNode { public StreamExecLookupJoin( + ReadableConfig tableConfig, FlinkJoinType joinType, @Nullable RexNode joinCondition, TemporalTableSourceSpec temporalTableSourceSpec, @@ -61,6 +63,7 @@ public StreamExecLookupJoin( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecLookupJoin.class), + ExecNodeContext.newPersistedConfig(StreamExecLookupJoin.class, tableConfig), joinType, joinCondition, temporalTableSourceSpec, @@ -76,6 +79,7 @@ public StreamExecLookupJoin( public StreamExecLookupJoin( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_JOIN_TYPE) FlinkJoinType joinType, @JsonProperty(FIELD_NAME_JOIN_CONDITION) @Nullable RexNode joinCondition, @JsonProperty(FIELD_NAME_TEMPORAL_TABLE) @@ -91,6 +95,7 @@ public StreamExecLookupJoin( super( id, context, + persistedConfig, joinType, joinCondition, temporalTableSourceSpec, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMatch.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMatch.java index 58f67116ecaa1..74b12c5050b20 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMatch.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMatch.java @@ -111,6 +111,7 @@ public class StreamExecMatch extends ExecNodeBase private final MatchSpec matchSpec; public StreamExecMatch( + ReadableConfig tableConfig, MatchSpec matchSpec, InputProperty inputProperty, RowType outputType, @@ -118,6 +119,7 @@ public StreamExecMatch( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecMatch.class), + ExecNodeContext.newPersistedConfig(StreamExecMatch.class, tableConfig), matchSpec, Collections.singletonList(inputProperty), outputType, @@ -128,11 +130,12 @@ public StreamExecMatch( public StreamExecMatch( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_MATCH_SPEC) MatchSpec matchSpec, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.matchSpec = checkNotNull(matchSpec); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMiniBatchAssigner.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMiniBatchAssigner.java index 2bbc423214d59..e2a0da4e9c197 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMiniBatchAssigner.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMiniBatchAssigner.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.RowData; @@ -73,6 +74,7 @@ public class StreamExecMiniBatchAssigner extends ExecNodeBase private final MiniBatchInterval miniBatchInterval; public StreamExecMiniBatchAssigner( + ReadableConfig tableConfig, MiniBatchInterval miniBatchInterval, InputProperty inputProperty, RowType outputType, @@ -80,6 +82,7 @@ public StreamExecMiniBatchAssigner( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecMiniBatchAssigner.class), + ExecNodeContext.newPersistedConfig(StreamExecMiniBatchAssigner.class, tableConfig), miniBatchInterval, Collections.singletonList(inputProperty), outputType, @@ -90,11 +93,12 @@ public StreamExecMiniBatchAssigner( public StreamExecMiniBatchAssigner( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_MINI_BATCH_INTERVAL) MiniBatchInterval miniBatchInterval, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); this.miniBatchInterval = checkNotNull(miniBatchInterval); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMultipleInput.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMultipleInput.java index b230f3cde2111..278feee81bd44 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMultipleInput.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMultipleInput.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; @@ -62,10 +63,14 @@ public class StreamExecMultipleInput extends ExecNodeBase private final ExecNode rootNode; public StreamExecMultipleInput( - List inputProperties, ExecNode rootNode, String description) { + ReadableConfig tableConfig, + List inputProperties, + ExecNode rootNode, + String description) { super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecMultipleInput.class), + ExecNodeContext.newPersistedConfig(StreamExecMultipleInput.class, tableConfig), inputProperties, rootNode.getOutputType(), description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java index 0a189f9853b29..5eaf5b933739d 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.functions.KeyedProcessFunction; import org.apache.flink.streaming.api.operators.KeyedProcessOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -44,6 +45,7 @@ import org.apache.flink.table.planner.plan.utils.KeySelectorUtil; import org.apache.flink.table.planner.plan.utils.OverAggregateUtil; import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; +import org.apache.flink.table.planner.utils.TableConfigUtils; import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction; import org.apache.flink.table.runtime.keyselector.RowDataKeySelector; import org.apache.flink.table.runtime.operators.over.ProcTimeRangeBoundedPrecedingFunction; @@ -83,7 +85,6 @@ @ExecNodeMetadata( name = "stream-exec-over-aggregate", version = 1, - consumedOptions = {"table.exec.state.ttl"}, producedTransformations = StreamExecOverAggregate.OVER_AGGREGATE_TRANSFORMATION, minPlanVersion = FlinkVersion.v1_15, minStateVersion = FlinkVersion.v1_15) @@ -100,6 +101,7 @@ public class StreamExecOverAggregate extends ExecNodeBase private final OverSpec overSpec; public StreamExecOverAggregate( + ReadableConfig tableConfig, OverSpec overSpec, InputProperty inputProperty, RowType outputType, @@ -107,6 +109,7 @@ public StreamExecOverAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecOverAggregate.class), + ExecNodeContext.newPersistedConfig(StreamExecOverAggregate.class, tableConfig), overSpec, Collections.singletonList(inputProperty), outputType, @@ -117,11 +120,12 @@ public StreamExecOverAggregate( public StreamExecOverAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_OVER_SPEC) OverSpec overSpec, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.overSpec = checkNotNull(overSpec); } @@ -307,7 +311,7 @@ private KeyedProcessFunction createUnboundedOverProce // ROWS unbounded over process function return new RowTimeRowsUnboundedPrecedingFunction<>( config.getStateRetentionTime(), - config.getMaxIdleStateRetentionTime(), + TableConfigUtils.getMaxIdleStateRetentionTime(config), genAggsHandler, flattenAccTypes, fieldTypes, @@ -316,7 +320,7 @@ private KeyedProcessFunction createUnboundedOverProce // RANGE unbounded over process function return new RowTimeRangeUnboundedPrecedingFunction<>( config.getStateRetentionTime(), - config.getMaxIdleStateRetentionTime(), + TableConfigUtils.getMaxIdleStateRetentionTime(config), genAggsHandler, flattenAccTypes, fieldTypes, @@ -325,7 +329,7 @@ private KeyedProcessFunction createUnboundedOverProce } else { return new ProcTimeUnboundedPrecedingFunction<>( config.getStateRetentionTime(), - config.getMaxIdleStateRetentionTime(), + TableConfigUtils.getMaxIdleStateRetentionTime(config), genAggsHandler, flattenAccTypes); } @@ -393,7 +397,7 @@ private KeyedProcessFunction createBoundedOverProcess if (isRowsClause) { return new RowTimeRowsBoundedPrecedingFunction<>( config.getStateRetentionTime(), - config.getMaxIdleStateRetentionTime(), + TableConfigUtils.getMaxIdleStateRetentionTime(config), genAggsHandler, flattenAccTypes, fieldTypes, @@ -407,7 +411,7 @@ private KeyedProcessFunction createBoundedOverProcess if (isRowsClause) { return new ProcTimeRowsBoundedPrecedingFunction<>( config.getStateRetentionTime(), - config.getMaxIdleStateRetentionTime(), + TableConfigUtils.getMaxIdleStateRetentionTime(config), genAggsHandler, flattenAccTypes, fieldTypes, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonCalc.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonCalc.java index 211221dd356b6..496ca96c1eb9a 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonCalc.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonCalc.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -34,6 +35,7 @@ public class StreamExecPythonCalc extends CommonExecPythonCalc implements StreamExecNode { public StreamExecPythonCalc( + ReadableConfig tableConfig, List projection, InputProperty inputProperty, RowType outputType, @@ -41,6 +43,7 @@ public StreamExecPythonCalc( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecPythonCalc.class), + ExecNodeContext.newPersistedConfig(StreamExecPythonCalc.class, tableConfig), projection, Collections.singletonList(inputProperty), outputType, @@ -50,10 +53,11 @@ public StreamExecPythonCalc( public StreamExecPythonCalc( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List projection, List inputProperties, RowType outputType, String description) { - super(id, context, projection, inputProperties, outputType, description); + super(id, context, persistedConfig, projection, inputProperties, outputType, description); } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonCorrelate.java index 992445cf9de05..0ce687a804c47 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonCorrelate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonCorrelate.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; import org.apache.flink.table.planner.plan.nodes.exec.InputProperty; @@ -36,6 +37,7 @@ public class StreamExecPythonCorrelate extends CommonExecPythonCorrelate implements StreamExecNode { public StreamExecPythonCorrelate( + ReadableConfig tableConfig, FlinkJoinType joinType, RexCall invocation, InputProperty inputProperty, @@ -44,6 +46,7 @@ public StreamExecPythonCorrelate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecPythonCorrelate.class), + ExecNodeContext.newPersistedConfig(StreamExecPythonCorrelate.class, tableConfig), joinType, invocation, Collections.singletonList(inputProperty), @@ -54,6 +57,7 @@ public StreamExecPythonCorrelate( public StreamExecPythonCorrelate( int id, ExecNodeContext context, + ReadableConfig persistedConfig, FlinkJoinType joinType, RexNode invocation, List inputProperties, @@ -62,6 +66,7 @@ public StreamExecPythonCorrelate( super( id, context, + persistedConfig, joinType, (RexCall) invocation, inputProperties, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupAggregate.java index d346e47447f6c..0d1cb1da282a5 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupAggregate.java @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -39,6 +40,7 @@ import org.apache.flink.table.planner.plan.utils.AggregateUtil; import org.apache.flink.table.planner.plan.utils.KeySelectorUtil; import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; +import org.apache.flink.table.planner.utils.TableConfigUtils; import org.apache.flink.table.runtime.dataview.DataViewSpec; import org.apache.flink.table.runtime.keyselector.RowDataKeySelector; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; @@ -77,6 +79,7 @@ public class StreamExecPythonGroupAggregate extends StreamExecAggregateBase { private final boolean needRetraction; public StreamExecPythonGroupAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, boolean[] aggCallNeedRetractions, @@ -88,6 +91,8 @@ public StreamExecPythonGroupAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecPythonGroupAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecPythonGroupAggregate.class, tableConfig), grouping, aggCalls, aggCallNeedRetractions, @@ -102,6 +107,7 @@ public StreamExecPythonGroupAggregate( public StreamExecPythonGroupAggregate( int id, ExecNodeContext context, + ReadableConfig persistedConfig, int[] grouping, AggregateCall[] aggCalls, boolean[] aggCallNeedRetractions, @@ -110,7 +116,7 @@ public StreamExecPythonGroupAggregate( List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); this.grouping = checkNotNull(grouping); this.aggCalls = checkNotNull(aggCalls); this.aggCallNeedRetractions = checkNotNull(aggCallNeedRetractions); @@ -151,7 +157,7 @@ protected Transformation translateToPlanInternal( PythonAggregateFunctionInfo[] pythonFunctionInfos = aggInfosAndDataViewSpecs.f0; DataViewSpec[][] dataViewSpecs = aggInfosAndDataViewSpecs.f1; Configuration pythonConfig = - CommonPythonUtil.getMergedConfig(planner.getExecEnv(), config.getTableConfig()); + CommonPythonUtil.extractPythonConfiguration(planner.getExecEnv(), config); final OneInputStreamOperator operator = getPythonAggregateFunctionOperator( pythonConfig, @@ -160,7 +166,7 @@ protected Transformation translateToPlanInternal( pythonFunctionInfos, dataViewSpecs, config.getStateRetentionTime(), - config.getMaxIdleStateRetentionTime(), + TableConfigUtils.getMaxIdleStateRetentionTime(config), inputCountIndex, countStarInserted); // partitioned aggregation diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupTableAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupTableAggregate.java index 40a1cf4473599..0b4cd55b824e7 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupTableAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupTableAggregate.java @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -41,6 +42,7 @@ import org.apache.flink.table.planner.plan.utils.AggregateUtil; import org.apache.flink.table.planner.plan.utils.KeySelectorUtil; import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; +import org.apache.flink.table.planner.utils.TableConfigUtils; import org.apache.flink.table.runtime.dataview.DataViewSpec; import org.apache.flink.table.runtime.keyselector.RowDataKeySelector; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; @@ -73,6 +75,7 @@ public class StreamExecPythonGroupTableAggregate extends ExecNodeBase private final boolean needRetraction; public StreamExecPythonGroupTableAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, boolean[] aggCallNeedRetractions, @@ -84,6 +87,8 @@ public StreamExecPythonGroupTableAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecPythonGroupTableAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecPythonGroupTableAggregate.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); @@ -125,7 +130,7 @@ protected Transformation translateToPlanInternal( PythonAggregateFunctionInfo[] pythonFunctionInfos = aggInfosAndDataViewSpecs.f0; DataViewSpec[][] dataViewSpecs = aggInfosAndDataViewSpecs.f1; Configuration pythonConfig = - CommonPythonUtil.getMergedConfig(planner.getExecEnv(), config.getTableConfig()); + CommonPythonUtil.extractPythonConfiguration(planner.getExecEnv(), config); OneInputStreamOperator pythonOperator = getPythonTableAggregateFunctionOperator( pythonConfig, @@ -134,7 +139,7 @@ protected Transformation translateToPlanInternal( pythonFunctionInfos, dataViewSpecs, config.getStateRetentionTime(), - config.getMaxIdleStateRetentionTime(), + TableConfigUtils.getMaxIdleStateRetentionTime(config), generateUpdateBefore, inputCountIndex); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupWindowAggregate.java index ea153ddefecf5..ee0605e3af8ec 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupWindowAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupWindowAggregate.java @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -53,6 +54,7 @@ import org.apache.flink.table.planner.plan.utils.PythonUtil; import org.apache.flink.table.planner.plan.utils.WindowEmitStrategy; import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; +import org.apache.flink.table.planner.utils.TableConfigUtils; import org.apache.flink.table.runtime.dataview.DataViewSpec; import org.apache.flink.table.runtime.generated.GeneratedProjection; import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty; @@ -71,8 +73,6 @@ import org.apache.flink.table.runtime.util.TimeWindowUtil; import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; - import org.apache.calcite.rel.core.AggregateCall; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -131,6 +131,7 @@ public class StreamExecPythonGroupWindowAggregate extends StreamExecAggregateBas private final boolean generateUpdateBefore; public StreamExecPythonGroupWindowAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, LogicalWindow window, @@ -143,6 +144,8 @@ public StreamExecPythonGroupWindowAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecPythonGroupWindowAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecPythonGroupWindowAggregate.class, tableConfig), grouping, aggCalls, window, @@ -154,10 +157,10 @@ public StreamExecPythonGroupWindowAggregate( description); } - @JsonCreator public StreamExecPythonGroupWindowAggregate( int id, ExecNodeContext context, + ReadableConfig persistedConfig, int[] grouping, AggregateCall[] aggCalls, LogicalWindow window, @@ -167,7 +170,7 @@ public StreamExecPythonGroupWindowAggregate( List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.grouping = checkNotNull(grouping); this.aggCalls = checkNotNull(aggCalls); @@ -224,13 +227,13 @@ protected Transformation translateToPlanInternal( final ZoneId shiftTimeZone = TimeWindowUtil.getShiftTimeZone( window.timeAttribute().getOutputDataType().getLogicalType(), - config.getLocalTimeZone()); + TableConfigUtils.getLocalTimeZone(config)); Tuple2, Trigger> windowAssignerAndTrigger = generateWindowAssignerAndTrigger(); WindowAssigner windowAssigner = windowAssignerAndTrigger.f0; Trigger trigger = windowAssignerAndTrigger.f1; Configuration pythonConfig = - CommonPythonUtil.getMergedConfig(planner.getExecEnv(), config.getTableConfig()); + CommonPythonUtil.extractPythonConfiguration(planner.getExecEnv(), config); boolean isGeneralPythonUDAF = Arrays.stream(aggCalls) .anyMatch(x -> PythonUtil.isPythonAggregate(x, PythonFunctionKind.GENERAL)); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonOverAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonOverAggregate.java index efc844bce0153..671fda6d848e4 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonOverAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonOverAggregate.java @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -43,14 +44,13 @@ import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil; import org.apache.flink.table.planner.plan.utils.KeySelectorUtil; import org.apache.flink.table.planner.plan.utils.OverAggregateUtil; +import org.apache.flink.table.planner.utils.TableConfigUtils; import org.apache.flink.table.runtime.generated.GeneratedProjection; import org.apache.flink.table.runtime.keyselector.RowDataKeySelector; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; - import org.apache.calcite.rel.core.AggregateCall; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -92,6 +92,7 @@ public class StreamExecPythonOverAggregate extends ExecNodeBase private final OverSpec overSpec; public StreamExecPythonOverAggregate( + ReadableConfig tableConfig, OverSpec overSpec, InputProperty inputProperty, RowType outputType, @@ -99,21 +100,23 @@ public StreamExecPythonOverAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecPythonOverAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecPythonOverAggregate.class, tableConfig), overSpec, Collections.singletonList(inputProperty), outputType, description); } - @JsonCreator public StreamExecPythonOverAggregate( int id, ExecNodeContext context, + ReadableConfig persistedConfig, OverSpec overSpec, List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.overSpec = checkNotNull(overSpec); } @@ -176,7 +179,7 @@ protected Transformation translateToPlanInternal( } long precedingOffset = -1 * (long) boundValue; Configuration pythonConfig = - CommonPythonUtil.getMergedConfig(planner.getExecEnv(), config.getTableConfig()); + CommonPythonUtil.extractPythonConfiguration(planner.getExecEnv(), config); OneInputTransformation transform = createPythonOneInputTransformation( inputTransform, @@ -187,7 +190,7 @@ protected Transformation translateToPlanInternal( precedingOffset, group.isRows(), config.getStateRetentionTime(), - config.getMaxIdleStateRetentionTime(), + TableConfigUtils.getMaxIdleStateRetentionTime(config), pythonConfig, config); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecRank.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecRank.java index bcb3f9c8f3430..1cb802ded8f0e 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecRank.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecRank.java @@ -21,6 +21,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.common.state.StateTtlConfig; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.KeyedProcessOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.table.api.TableException; @@ -75,7 +76,7 @@ @ExecNodeMetadata( name = "stream-exec-rank", version = 1, - consumedOptions = {"table.exec.state.ttl", "table.exec.rank.topn-cache-size"}, + consumedOptions = {"table.exec.rank.topn-cache-size"}, producedTransformations = StreamExecRank.RANK_TRANSFORMATION, minPlanVersion = FlinkVersion.v1_15, minStateVersion = FlinkVersion.v1_15) @@ -114,6 +115,7 @@ public class StreamExecRank extends ExecNodeBase private final boolean generateUpdateBefore; public StreamExecRank( + ReadableConfig tableConfig, RankType rankType, PartitionSpec partitionSpec, SortSpec sortSpec, @@ -127,6 +129,7 @@ public StreamExecRank( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecRank.class), + ExecNodeContext.newPersistedConfig(StreamExecRank.class, tableConfig), rankType, partitionSpec, sortSpec, @@ -143,6 +146,7 @@ public StreamExecRank( public StreamExecRank( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_RANK_TYPE) RankType rankType, @JsonProperty(FIELD_NAME_PARTITION_SPEC) PartitionSpec partitionSpec, @JsonProperty(FIELD_NAME_SORT_SPEC) SortSpec sortSpec, @@ -153,7 +157,7 @@ public StreamExecRank( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.rankType = checkNotNull(rankType); this.rankRange = checkNotNull(rankRange); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSink.java index 23341599f3db9..40acb2f964222 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSink.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSink.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.sink.DynamicTableSink; @@ -55,7 +56,6 @@ name = "stream-exec-sink", version = 1, consumedOptions = { - "table.exec.state.ttl", "table.exec.sink.not-null-enforcer", "table.exec.sink.type-length-enforcer", "table.exec.sink.upsert-materialize", @@ -83,6 +83,7 @@ public class StreamExecSink extends CommonExecSink implements StreamExecNode inputProperties, @@ -113,6 +116,7 @@ public StreamExecSink( super( id, context, + persistedConfig, tableSinkSpec, inputChangelogMode, false, // isBounded diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSort.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSort.java index 6d5424c0e2757..cf8e112b43e67 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSort.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSort.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.sort.ComparatorCodeGenerator; @@ -51,6 +52,7 @@ public class StreamExecSort extends ExecNodeBase implements StreamExecN private final SortSpec sortSpec; public StreamExecSort( + ReadableConfig tableConfig, SortSpec sortSpec, InputProperty inputProperty, RowType outputType, @@ -58,6 +60,7 @@ public StreamExecSort( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecSort.class), + ExecNodeContext.newPersistedConfig(StreamExecSort.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSortLimit.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSortLimit.java index 3c2cf3acf89fe..fd36729592264 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSortLimit.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSortLimit.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; @@ -44,7 +45,7 @@ @ExecNodeMetadata( name = "stream-exec-sort-limit", version = 1, - consumedOptions = {"table.exec.state.ttl", "table.exec.rank.topn-cache-size"}, + consumedOptions = {"table.exec.rank.topn-cache-size"}, producedTransformations = StreamExecRank.RANK_TRANSFORMATION, minPlanVersion = FlinkVersion.v1_15, minStateVersion = FlinkVersion.v1_15) @@ -53,6 +54,7 @@ public class StreamExecSortLimit extends StreamExecRank { private final long limitEnd; public StreamExecSortLimit( + ReadableConfig tableConfig, SortSpec sortSpec, long limitStart, long limitEnd, @@ -64,6 +66,7 @@ public StreamExecSortLimit( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecSortLimit.class), + ExecNodeContext.newPersistedConfig(StreamExecSortLimit.class, tableConfig), sortSpec, new ConstantRankRange(limitStart + 1, limitEnd), rankStrategy, @@ -77,6 +80,7 @@ public StreamExecSortLimit( public StreamExecSortLimit( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_SORT_SPEC) SortSpec sortSpec, @JsonProperty(FIELD_NAME_RANK_RANG) ConstantRankRange rankRange, @JsonProperty(FIELD_NAME_RANK_STRATEGY) RankProcessStrategy rankStrategy, @@ -88,6 +92,7 @@ public StreamExecSortLimit( super( id, context, + persistedConfig, RankType.ROW_NUMBER, PartitionSpec.ALL_IN_ONE, sortSpec, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTableSourceScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTableSourceScan.java index d7f642baf65c7..9fb292bd96030 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTableSourceScan.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTableSourceScan.java @@ -21,6 +21,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.connector.source.ScanTableSource; import org.apache.flink.table.data.RowData; @@ -49,10 +50,14 @@ public class StreamExecTableSourceScan extends CommonExecTableSourceScan implements StreamExecNode { public StreamExecTableSourceScan( - DynamicTableSourceSpec tableSourceSpec, RowType outputType, String description) { + ReadableConfig tableConfig, + DynamicTableSourceSpec tableSourceSpec, + RowType outputType, + String description) { this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecTableSourceScan.class), + ExecNodeContext.newPersistedConfig(StreamExecTableSourceScan.class, tableConfig), tableSourceSpec, outputType, description); @@ -62,10 +67,11 @@ public StreamExecTableSourceScan( public StreamExecTableSourceScan( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_SCAN_TABLE_SOURCE) DynamicTableSourceSpec tableSourceSpec, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, tableSourceSpec, outputType, description); + super(id, context, persistedConfig, tableSourceSpec, outputType, description); } @Override diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalJoin.java index 11125be16d177..6797ec8406fb6 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalJoin.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; import org.apache.flink.streaming.api.transformations.TwoInputTransformation; import org.apache.flink.table.api.TableException; @@ -43,6 +44,7 @@ import org.apache.flink.table.planner.plan.utils.JoinUtil; import org.apache.flink.table.planner.plan.utils.KeySelectorUtil; import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; +import org.apache.flink.table.planner.utils.TableConfigUtils; import org.apache.flink.table.runtime.generated.GeneratedJoinCondition; import org.apache.flink.table.runtime.keyselector.RowDataKeySelector; import org.apache.flink.table.runtime.operators.join.FlinkJoinType; @@ -69,7 +71,6 @@ @ExecNodeMetadata( name = "stream-exec-temporal-join", version = 1, - consumedOptions = "table.exec.state.ttl", producedTransformations = StreamExecTemporalJoin.TEMPORAL_JOIN_TRANSFORMATION, minPlanVersion = FlinkVersion.v1_15, minStateVersion = FlinkVersion.v1_15) @@ -97,6 +98,7 @@ public class StreamExecTemporalJoin extends ExecNodeBase private final int rightTimeAttributeIndex; public StreamExecTemporalJoin( + ReadableConfig tableConfig, JoinSpec joinSpec, boolean isTemporalTableFunctionJoin, int leftTimeAttributeIndex, @@ -108,6 +110,7 @@ public StreamExecTemporalJoin( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecTemporalJoin.class), + ExecNodeContext.newPersistedConfig(StreamExecTemporalJoin.class, tableConfig), joinSpec, isTemporalTableFunctionJoin, leftTimeAttributeIndex, @@ -121,6 +124,7 @@ public StreamExecTemporalJoin( public StreamExecTemporalJoin( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_JOIN_SPEC) JoinSpec joinSpec, @JsonProperty(FIELD_NAME_IS_TEMPORAL_FUNCTION_JOIN) boolean isTemporalTableFunctionJoin, @JsonProperty(FIELD_NAME_LEFT_TIME_ATTRIBUTE_INDEX) int leftTimeAttributeIndex, @@ -128,7 +132,7 @@ public StreamExecTemporalJoin( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); Preconditions.checkArgument(inputProperties.size() == 2); Preconditions.checkArgument( rightTimeAttributeIndex == FIELD_INDEX_FOR_PROC_TIME_ATTRIBUTE @@ -248,7 +252,7 @@ private TwoInputStreamOperator createJoinOperator( boolean isLeftOuterJoin = joinSpec.getJoinType() == FlinkJoinType.LEFT; long minRetentionTime = config.getStateRetentionTime(); - long maxRetentionTime = config.getMaxIdleStateRetentionTime(); + long maxRetentionTime = TableConfigUtils.getMaxIdleStateRetentionTime(config); if (rightTimeAttributeIndex >= 0) { return new TemporalRowTimeJoinOperator( InternalTypeInfo.of(leftInputType), diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalSort.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalSort.java index ed5d49c4827f2..c0c88a511b06c 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalSort.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalSort.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.RowData; @@ -71,6 +72,7 @@ public class StreamExecTemporalSort extends ExecNodeBase private final SortSpec sortSpec; public StreamExecTemporalSort( + ReadableConfig tableConfig, SortSpec sortSpec, InputProperty inputProperty, RowType outputType, @@ -78,6 +80,7 @@ public StreamExecTemporalSort( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecTemporalSort.class), + ExecNodeContext.newPersistedConfig(StreamExecTemporalSort.class, tableConfig), sortSpec, Collections.singletonList(inputProperty), outputType, @@ -88,11 +91,12 @@ public StreamExecTemporalSort( public StreamExecTemporalSort( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_SORT_SPEC) SortSpec sortSpec, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.sortSpec = checkNotNull(sortSpec); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecUnion.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecUnion.java index 68c9f9b4aa20b..0cfbde1fb938f 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecUnion.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecUnion.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.FlinkVersion; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -44,10 +45,14 @@ public class StreamExecUnion extends CommonExecUnion implements StreamExecNode { public StreamExecUnion( - List inputProperties, RowType outputType, String description) { + ReadableConfig tableConfig, + List inputProperties, + RowType outputType, + String description) { this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecUnion.class), + ExecNodeContext.newPersistedConfig(StreamExecUnion.class, tableConfig), inputProperties, outputType, description); @@ -57,9 +62,10 @@ public StreamExecUnion( public StreamExecUnion( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecValues.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecValues.java index 3b6c984d51614..9ace41eaf896c 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecValues.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecValues.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.FlinkVersion; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -42,10 +43,15 @@ minStateVersion = FlinkVersion.v1_15) public class StreamExecValues extends CommonExecValues implements StreamExecNode { - public StreamExecValues(List> tuples, RowType outputType, String description) { + public StreamExecValues( + ReadableConfig tableConfig, + List> tuples, + RowType outputType, + String description) { this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecValues.class), + ExecNodeContext.newPersistedConfig(StreamExecValues.class, tableConfig), tuples, outputType, description); @@ -55,9 +61,10 @@ public StreamExecValues(List> tuples, RowType outputType, Strin public StreamExecValues( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_TUPLES) List> tuples, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, tuples, outputType, description); + super(id, context, persistedConfig, tuples, outputType, description); } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWatermarkAssigner.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWatermarkAssigner.java index a27c3ff6149ee..f79488a05d5df 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWatermarkAssigner.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWatermarkAssigner.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.WatermarkGeneratorCodeGenerator; @@ -55,7 +56,6 @@ @ExecNodeMetadata( name = "stream-exec-watermark-assigner", version = 1, - consumedOptions = "table.exec.source.idle-timeout", producedTransformations = StreamExecWatermarkAssigner.WATERMARK_ASSIGNER_TRANSFORMATION, minPlanVersion = FlinkVersion.v1_15, minStateVersion = FlinkVersion.v1_15) @@ -74,6 +74,7 @@ public class StreamExecWatermarkAssigner extends ExecNodeBase private final int rowtimeFieldIndex; public StreamExecWatermarkAssigner( + ReadableConfig tableConfig, RexNode watermarkExpr, int rowtimeFieldIndex, InputProperty inputProperty, @@ -82,6 +83,7 @@ public StreamExecWatermarkAssigner( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecWatermarkAssigner.class), + ExecNodeContext.newPersistedConfig(StreamExecWatermarkAssigner.class, tableConfig), watermarkExpr, rowtimeFieldIndex, Collections.singletonList(inputProperty), @@ -93,12 +95,13 @@ public StreamExecWatermarkAssigner( public StreamExecWatermarkAssigner( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_WATERMARK_EXPR) RexNode watermarkExpr, @JsonProperty(FIELD_NAME_ROWTIME_FIELD_INDEX) int rowtimeFieldIndex, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.watermarkExpr = checkNotNull(watermarkExpr); this.rowtimeFieldIndex = rowtimeFieldIndex; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java index 094c70c578af0..24fc63821be71 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -39,6 +40,7 @@ import org.apache.flink.table.planner.plan.utils.AggregateUtil; import org.apache.flink.table.planner.plan.utils.KeySelectorUtil; import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; +import org.apache.flink.table.planner.utils.TableConfigUtils; import org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction; import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty; import org.apache.flink.table.runtime.groupwindow.WindowProperty; @@ -103,6 +105,7 @@ public class StreamExecWindowAggregate extends StreamExecWindowAggregateBase { private final NamedWindowProperty[] namedWindowProperties; public StreamExecWindowAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, WindowingStrategy windowing, @@ -113,6 +116,7 @@ public StreamExecWindowAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecWindowAggregate.class), + ExecNodeContext.newPersistedConfig(StreamExecWindowAggregate.class, tableConfig), grouping, aggCalls, windowing, @@ -126,6 +130,7 @@ public StreamExecWindowAggregate( public StreamExecWindowAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_GROUPING) int[] grouping, @JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls, @JsonProperty(FIELD_NAME_WINDOWING) WindowingStrategy windowing, @@ -134,7 +139,7 @@ public StreamExecWindowAggregate( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); this.grouping = checkNotNull(grouping); this.aggCalls = checkNotNull(aggCalls); this.windowing = checkNotNull(windowing); @@ -152,7 +157,8 @@ protected Transformation translateToPlanInternal( final ZoneId shiftTimeZone = TimeWindowUtil.getShiftTimeZone( - windowing.getTimeAttributeType(), config.getLocalTimeZone()); + windowing.getTimeAttributeType(), + TableConfigUtils.getLocalTimeZone(config)); final SliceAssigner sliceAssigner = createSliceAssigner(windowing, shiftTimeZone); // Hopping window requires additional COUNT(*) to determine whether to register next timer diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregateBase.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregateBase.java index 59adf5531b484..b9a01a041f269 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregateBase.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregateBase.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.planner.plan.logical.CumulativeWindowSpec; import org.apache.flink.table.planner.plan.logical.HoppingWindowSpec; @@ -53,10 +54,11 @@ public abstract class StreamExecWindowAggregateBase extends StreamExecAggregateB protected StreamExecWindowAggregateBase( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List inputProperties, LogicalType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowDeduplicate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowDeduplicate.java index b0ddee099b1da..4f8dd6087e0ca 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowDeduplicate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowDeduplicate.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -38,6 +39,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator; import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil; import org.apache.flink.table.planner.plan.utils.KeySelectorUtil; +import org.apache.flink.table.planner.utils.TableConfigUtils; import org.apache.flink.table.runtime.keyselector.RowDataKeySelector; import org.apache.flink.table.runtime.operators.deduplicate.window.RowTimeWindowDeduplicateOperatorBuilder; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; @@ -89,6 +91,7 @@ public class StreamExecWindowDeduplicate extends ExecNodeBase private final WindowingStrategy windowing; public StreamExecWindowDeduplicate( + ReadableConfig tableConfig, int[] partitionKeys, int orderKey, boolean keepLastRow, @@ -99,6 +102,7 @@ public StreamExecWindowDeduplicate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecWindowDeduplicate.class), + ExecNodeContext.newPersistedConfig(StreamExecWindowDeduplicate.class, tableConfig), partitionKeys, orderKey, keepLastRow, @@ -112,6 +116,7 @@ public StreamExecWindowDeduplicate( public StreamExecWindowDeduplicate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_PARTITION_KEYS) int[] partitionKeys, @JsonProperty(FIELD_NAME_ORDER_KEY) int orderKey, @JsonProperty(FIELD_NAME_KEEP_LAST_ROW) boolean keepLastRow, @@ -119,7 +124,7 @@ public StreamExecWindowDeduplicate( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.partitionKeys = checkNotNull(partitionKeys); this.orderKey = orderKey; @@ -149,7 +154,8 @@ protected Transformation translateToPlanInternal( ZoneId shiftTimeZone = TimeWindowUtil.getShiftTimeZone( - windowing.getTimeAttributeType(), config.getLocalTimeZone()); + windowing.getTimeAttributeType(), + TableConfigUtils.getLocalTimeZone(config)); RowType inputType = (RowType) inputEdge.getOutputType(); RowDataKeySelector selector = diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowJoin.java index e87cb3f9f3f22..750d822cce902 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowJoin.java @@ -21,6 +21,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.transformations.TwoInputTransformation; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.RowData; @@ -38,6 +39,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil; import org.apache.flink.table.planner.plan.utils.JoinUtil; import org.apache.flink.table.planner.plan.utils.KeySelectorUtil; +import org.apache.flink.table.planner.utils.TableConfigUtils; import org.apache.flink.table.runtime.generated.GeneratedJoinCondition; import org.apache.flink.table.runtime.keyselector.RowDataKeySelector; import org.apache.flink.table.runtime.operators.join.window.WindowJoinOperator; @@ -83,6 +85,7 @@ public class StreamExecWindowJoin extends ExecNodeBase private final WindowingStrategy rightWindowing; public StreamExecWindowJoin( + ReadableConfig tableConfig, JoinSpec joinSpec, WindowingStrategy leftWindowing, WindowingStrategy rightWindowing, @@ -93,6 +96,7 @@ public StreamExecWindowJoin( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecWindowJoin.class), + ExecNodeContext.newPersistedConfig(StreamExecWindowJoin.class, tableConfig), joinSpec, leftWindowing, rightWindowing, @@ -105,13 +109,14 @@ public StreamExecWindowJoin( public StreamExecWindowJoin( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_JOIN_SPEC) JoinSpec joinSpec, @JsonProperty(FIELD_NAME_LEFT_WINDOWING) WindowingStrategy leftWindowing, @JsonProperty(FIELD_NAME_RIGHT_WINDOWING) WindowingStrategy rightWindowing, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 2); this.joinSpec = checkNotNull(joinSpec); validate(leftWindowing); @@ -161,7 +166,8 @@ protected Transformation translateToPlanInternal( ZoneId shiftTimeZone = TimeWindowUtil.getShiftTimeZone( - leftWindowing.getTimeAttributeType(), config.getLocalTimeZone()); + leftWindowing.getTimeAttributeType(), + TableConfigUtils.getLocalTimeZone(config)); WindowJoinOperator operator = WindowJoinOperatorBuilder.builder() .leftSerializer(leftTypeInfo.toRowSerializer()) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowRank.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowRank.java index 73f752ced3789..b6911385e4ca5 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowRank.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowRank.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -41,6 +42,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.spec.SortSpec; import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil; import org.apache.flink.table.planner.plan.utils.KeySelectorUtil; +import org.apache.flink.table.planner.utils.TableConfigUtils; import org.apache.flink.table.runtime.generated.GeneratedRecordComparator; import org.apache.flink.table.runtime.keyselector.RowDataKeySelector; import org.apache.flink.table.runtime.operators.rank.ConstantRankRange; @@ -105,6 +107,7 @@ public class StreamExecWindowRank extends ExecNodeBase private final WindowingStrategy windowing; public StreamExecWindowRank( + ReadableConfig tableConfig, RankType rankType, PartitionSpec partitionSpec, SortSpec sortSpec, @@ -117,6 +120,7 @@ public StreamExecWindowRank( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecWindowRank.class), + ExecNodeContext.newPersistedConfig(StreamExecWindowRank.class, tableConfig), rankType, partitionSpec, sortSpec, @@ -132,6 +136,7 @@ public StreamExecWindowRank( public StreamExecWindowRank( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_RANK_TYPE) RankType rankType, @JsonProperty(FIELD_NAME_PARTITION_SPEC) PartitionSpec partitionSpec, @JsonProperty(FIELD_NAME_SORT_SPEC) SortSpec sortSpec, @@ -141,7 +146,7 @@ public StreamExecWindowRank( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.rankType = checkNotNull(rankType); this.partitionSpec = checkNotNull(partitionSpec); @@ -218,7 +223,8 @@ protected Transformation translateToPlanInternal( ZoneId shiftTimeZone = TimeWindowUtil.getShiftTimeZone( - windowing.getTimeAttributeType(), config.getLocalTimeZone()); + windowing.getTimeAttributeType(), + TableConfigUtils.getLocalTimeZone(config)); GeneratedRecordComparator sortKeyComparator = ComparatorCodeGenerator.gen( config.getTableConfig(), diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowTableFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowTableFunction.java index 38ea6ee89a2a0..7728712a6af29 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowTableFunction.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowTableFunction.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.FlinkVersion; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.logical.TimeAttributeWindowingStrategy; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; @@ -51,6 +52,7 @@ public class StreamExecWindowTableFunction extends CommonExecWindowTableFunction implements StreamExecNode { public StreamExecWindowTableFunction( + ReadableConfig tableConfig, TimeAttributeWindowingStrategy windowingStrategy, InputProperty inputProperty, RowType outputType, @@ -58,6 +60,8 @@ public StreamExecWindowTableFunction( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecWindowTableFunction.class), + ExecNodeContext.newPersistedConfig( + StreamExecWindowTableFunction.class, tableConfig), windowingStrategy, Collections.singletonList(inputProperty), outputType, @@ -68,10 +72,18 @@ public StreamExecWindowTableFunction( public StreamExecWindowTableFunction( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_WINDOWING) TimeAttributeWindowingStrategy windowingStrategy, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, windowingStrategy, inputProperties, outputType, description); + super( + id, + context, + persistedConfig, + windowingStrategy, + inputProperties, + outputType, + description); } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/utils/CommonPythonUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/utils/CommonPythonUtil.java index 538bfdb37ef6e..a949ad2afa6c3 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/utils/CommonPythonUtil.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/utils/CommonPythonUtil.java @@ -21,8 +21,8 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.dataview.DataView; import org.apache.flink.table.api.dataview.ListView; @@ -98,7 +98,7 @@ public class CommonPythonUtil { private CommonPythonUtil() {} - public static Class loadClass(String className) { + public static Class loadClass(String className) { try { return Class.forName(className, false, Thread.currentThread().getContextClassLoader()); } catch (ClassNotFoundException e) { @@ -107,21 +107,20 @@ public static Class loadClass(String className) { } } - @SuppressWarnings("unchecked") - public static Configuration getMergedConfig( - StreamExecutionEnvironment env, TableConfig tableConfig) { - Class clazz = loadClass(PYTHON_CONFIG_UTILS_CLASS); + public static Configuration extractPythonConfiguration( + StreamExecutionEnvironment env, ReadableConfig tableConfig) { + Class clazz = loadClass(PYTHON_CONFIG_UTILS_CLASS); try { StreamExecutionEnvironment realEnv = getRealEnvironment(env); Method method = clazz.getDeclaredMethod( - "getMergedConfig", StreamExecutionEnvironment.class, TableConfig.class); - return (Configuration) method.invoke(null, realEnv, tableConfig); + "extractPythonConfiguration", List.class, ReadableConfig.class); + return (Configuration) method.invoke(null, realEnv.getCachedFiles(), tableConfig); } catch (NoSuchFieldException | IllegalAccessException | NoSuchMethodException | InvocationTargetException e) { - throw new TableException("Method getMergedConfig accessed failed.", e); + throw new TableException("Method extractPythonConfiguration accessed failed.", e); } } @@ -149,7 +148,7 @@ public static PythonFunctionInfo createPythonFunctionInfo( @SuppressWarnings("unchecked") public static boolean isPythonWorkerUsingManagedMemory(Configuration config) { - Class clazz = loadClass(PYTHON_OPTIONS_CLASS); + Class clazz = loadClass(PYTHON_OPTIONS_CLASS); try { return config.getBoolean( (ConfigOption) (clazz.getField("USE_MANAGED_MEMORY").get(null))); @@ -160,7 +159,7 @@ public static boolean isPythonWorkerUsingManagedMemory(Configuration config) { @SuppressWarnings("unchecked") public static boolean isPythonWorkerInProcessMode(Configuration config) { - Class clazz = loadClass(PYTHON_OPTIONS_CLASS); + Class clazz = loadClass(PYTHON_OPTIONS_CLASS); try { return config.getString( (ConfigOption) @@ -401,12 +400,12 @@ private static byte[] convertLiteralToPython(RexLiteral o, SqlTypeName typeName) return (byte[]) pickleValue.invoke(null, value, type); } - @SuppressWarnings("unchecked") private static void loadPickleValue() { if (pickleValue == null) { synchronized (CommonPythonUtil.class) { if (pickleValue == null) { - Class clazz = loadClass("org.apache.flink.api.common.python.PythonBridgeUtils"); + Class clazz = + loadClass("org.apache.flink.api.common.python.PythonBridgeUtils"); try { pickleValue = clazz.getMethod("pickleValue", Object.class, byte.class); } catch (NoSuchMethodException e) { diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoSourceScanRuleBase.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoSourceScanRuleBase.java index e3203cf0d190d..c158b0bfcb77e 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoSourceScanRuleBase.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoSourceScanRuleBase.java @@ -58,10 +58,8 @@ public PushFilterIntoSourceScanRuleBase(RelOptRuleOperand operand, String descri @Override public boolean matches(RelOptRuleCall call) { TableConfig tableConfig = ShortcutUtils.unwrapContext(call.getPlanner()).getTableConfig(); - return tableConfig - .getConfiguration() - .getBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_SOURCE_PREDICATE_PUSHDOWN_ENABLED); + return tableConfig.get( + OptimizerConfigOptions.TABLE_OPTIMIZER_SOURCE_PREDICATE_PUSHDOWN_ENABLED); } protected List convertExpressionToRexNode( diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.java index a4044bf45b05d..17e87a4bd4581 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.java @@ -61,7 +61,7 @@ import java.util.stream.Stream; import static org.apache.flink.table.planner.connectors.DynamicSourceUtils.createProducedType; -import static org.apache.flink.table.planner.connectors.DynamicSourceUtils.createRequiredMetadataKeys; +import static org.apache.flink.table.planner.connectors.DynamicSourceUtils.createRequiredMetadataColumns; import static org.apache.flink.table.planner.utils.ShortcutUtils.unwrapContext; import static org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTypeFactory; @@ -249,12 +249,16 @@ private RowType performPushDown( final List projectedMetadataColumns; if (supportsMetadata(source.tableSource())) { final List declaredMetadataKeys = - createRequiredMetadataKeys( - source.contextResolvedTable().getResolvedSchema(), - source.tableSource()); + createRequiredMetadataColumns( + source.contextResolvedTable().getResolvedSchema(), + source.tableSource()) + .stream() + .map(col -> col.getMetadataKey().orElse(col.getName())) + .collect(Collectors.toList()); numPhysicalColumns = producedType.getFieldCount() - declaredMetadataKeys.size(); + // the projected metadata column name projectedMetadataColumns = IntStream.range(0, declaredMetadataKeys.size()) .mapToObj(i -> producedType.getFieldNames().get(numPhysicalColumns + i)) @@ -305,9 +309,23 @@ private RowType performPushDown( (RowType) Projection.of(projectedFields).project(producedType); if (supportsMetadata(source.tableSource())) { + // Use the projected column name to get the metadata key final List projectedMetadataKeys = projectedMetadataColumns.stream() - .map(NestedColumn::name) + .map( + nestedColumn -> + source.contextResolvedTable() + .getResolvedSchema() + .getColumn(nestedColumn.name()) + .orElseThrow( + () -> + new TableException( + String.format( + "Can not find the column %s in the origin schema.", + nestedColumn + .name())))) + .map(Column.MetadataColumn.class::cast) + .map(col -> col.getMetadataKey().orElse(col.getName())) .collect(Collectors.toList()); abilitySpecs.add(new ReadingMetadataSpec(projectedMetadataKeys, newProducedType)); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleBase.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleBase.java index 5d92e7cdc5964..06ee579c78705 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleBase.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleBase.java @@ -109,9 +109,7 @@ protected FlinkLogicalTableSourceScan getNewScan( abilitySpec = sourceWatermarkSpec; } else { final Duration idleTimeout = - tableConfig - .getConfiguration() - .get(ExecutionConfigOptions.TABLE_EXEC_SOURCE_IDLE_TIMEOUT); + tableConfig.get(ExecutionConfigOptions.TABLE_EXEC_SOURCE_IDLE_TIMEOUT); final long idleTimeoutMillis; if (!idleTimeout.isZero() && !idleTimeout.isNegative()) { idleTimeoutMillis = idleTimeout.toMillis(); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/SubQueryDecorrelator.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/SubQueryDecorrelator.java index b80b884da8b62..828286b5fc665 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/SubQueryDecorrelator.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/SubQueryDecorrelator.java @@ -134,8 +134,7 @@ public static Result decorrelateQuery(RelNode rootRel) { } RelOptCluster cluster = rootRel.getCluster(); - RelBuilder relBuilder = - new FlinkRelBuilder(cluster.getPlanner().getContext(), cluster, null); + RelBuilder relBuilder = FlinkRelBuilder.of(cluster, null); RexBuilder rexBuilder = cluster.getRexBuilder(); final SubQueryDecorrelator decorrelator = diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalPythonWindowAggregateRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalPythonWindowAggregateRule.java index 9eadb76c50ce6..847b803c1b9d8 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalPythonWindowAggregateRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalPythonWindowAggregateRule.java @@ -34,6 +34,7 @@ import org.apache.flink.table.planner.plan.utils.AggregateUtil; import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil; import org.apache.flink.table.planner.plan.utils.PythonUtil; +import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; import org.apache.flink.table.types.DataType; import org.apache.calcite.plan.RelOptRule; @@ -160,7 +161,7 @@ public void onMatch(RelOptRuleCall call) { window, inputTimeFieldIndex, inputTimeIsDate, - agg.getNamedProperties()); + JavaScalaConversionUtil.toScala(agg.getNamedProperties())); call.transformTo(windowAgg); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoScanRuleBase.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoScanRuleBase.java index fe9a8584b02aa..81bdaf68767ae 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoScanRuleBase.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoScanRuleBase.java @@ -78,10 +78,8 @@ protected boolean canPushDown( BatchPhysicalGroupAggregateBase aggregate, BatchPhysicalTableSourceScan tableSourceScan) { TableConfig tableConfig = ShortcutUtils.unwrapContext(call.getPlanner()).getTableConfig(); - if (!tableConfig - .getConfiguration() - .getBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_SOURCE_AGGREGATE_PUSHDOWN_ENABLED)) { + if (!tableConfig.get( + OptimizerConfigOptions.TABLE_OPTIMIZER_SOURCE_AGGREGATE_PUSHDOWN_ENABLED)) { return false; } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalPythonGroupWindowAggregateRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalPythonGroupWindowAggregateRule.java index 002f1ae59481d..5f080d76da7aa 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalPythonGroupWindowAggregateRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalPythonGroupWindowAggregateRule.java @@ -143,7 +143,7 @@ public RelNode convert(RelNode rel) { agg.getGroupSet().toArray(), JavaScalaConversionUtil.toScala(aggCalls), agg.getWindow(), - agg.getNamedProperties(), + JavaScalaConversionUtil.toScala(agg.getNamedProperties()), emitStrategy); } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.java index d86937bfd629a..385d930353539 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.java @@ -18,7 +18,6 @@ package org.apache.flink.table.planner.plan.schema; -import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.catalog.Catalog; @@ -118,7 +117,7 @@ public RelNode toRel(ToRelContext toRelContext) { // prepare table source and convert to RelNode return DynamicSourceUtils.convertSourceToRel( !schemaTable.isStreamingMode(), - context.getTableConfig().getConfiguration(), + context.getTableConfig(), relBuilder, schemaTable.getContextResolvedTable(), schemaTable.getStatistic(), @@ -132,8 +131,7 @@ private ContextResolvedTable computeContextResolvedTable( if (hintedOptions.isEmpty()) { return contextResolvedTable; } - final ReadableConfig config = context.getTableConfig().getConfiguration(); - if (!config.get(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED)) { + if (!context.getTableConfig().get(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED)) { throw new ValidationException( String.format( "The '%s' hint is allowed only when the config option '%s' is set to true.", @@ -154,7 +152,6 @@ private ContextResolvedTable computeContextResolvedTable( private DynamicTableSource createDynamicTableSource( FlinkContext context, ResolvedCatalogTable catalogTable) { - final ReadableConfig config = context.getTableConfig().getConfiguration(); final Optional factoryFromCatalog = schemaTable @@ -179,7 +176,7 @@ private DynamicTableSource createDynamicTableSource( factory, schemaTable.getContextResolvedTable().getIdentifier(), catalogTable, - config, + context.getTableConfig(), Thread.currentThread().getContextClassLoader(), schemaTable.isTemporary()); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java index c295e31482683..2da4fffb1c46f 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java @@ -20,6 +20,12 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigUtils; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.config.ExecutionConfigOptions; +import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata; import org.apache.flink.table.planner.plan.nodes.exec.MultipleExecNodeMetadata; @@ -73,7 +79,10 @@ import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowRank; import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowTableFunction; +import javax.annotation.Nullable; + import java.util.ArrayList; +import java.util.Arrays; import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; @@ -81,6 +90,7 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.stream.Stream; /** Utility class for {@link ExecNodeMetadata} related functionality. */ @Internal @@ -159,6 +169,18 @@ private ExecNodeMetadataUtil() { } }; + public static final Set> TABLE_CONFIG_OPTIONS; + + static { + TABLE_CONFIG_OPTIONS = ConfigUtils.getAllConfigOptions(TableConfigOptions.class); + } + + public static final Set> EXECUTION_CONFIG_OPTIONS; + + static { + EXECUTION_CONFIG_OPTIONS = ConfigUtils.getAllConfigOptions(ExecutionConfigOptions.class); + } + public static Set>> execNodes() { return EXEC_NODES; } @@ -254,6 +276,63 @@ public static > ExecNodeMetadata latestAnnotation( return sortedAnnotations.get(sortedAnnotations.size() - 1); } + @Nullable + public static > String[] consumedOptions(Class execNodeClass) { + ExecNodeMetadata metadata = latestAnnotation(execNodeClass); + if (metadata == null) { + return null; + } + return metadata.consumedOptions(); + } + + @SuppressWarnings({"rawtypes", "unchecked"}) + public static > ReadableConfig newPersistedConfig( + Class execNodeClass, + ReadableConfig tableConfig, + Stream> configOptions) { + final Map> availableConfigOptions = new HashMap<>(); + configOptions.forEach( + co -> { + availableConfigOptions.put(co.key(), co); + co.fallbackKeys().forEach(k -> availableConfigOptions.put(k.getKey(), co)); + }); + + final Configuration persistedConfig = new Configuration(); + final String[] consumedOptions = ExecNodeMetadataUtil.consumedOptions(execNodeClass); + if (consumedOptions == null) { + return persistedConfig; + } + + final Map nodeConfigOptions = new HashMap<>(); + for (final String consumedOption : consumedOptions) { + ConfigOption configOption = availableConfigOptions.get(consumedOption); + if (configOption == null) { + throw new IllegalStateException( + String.format( + "ExecNode: %s, consumedOption: %s not listed in [%s].", + execNodeClass.getCanonicalName(), + consumedOption, + String.join( + ", ", + Arrays.asList( + TableConfigOptions.class.getSimpleName(), + ExecutionConfigOptions.class.getSimpleName())))); + } + if (nodeConfigOptions.containsKey(configOption)) { + throw new IllegalStateException( + String.format( + "ExecNode: %s, consumedOption: %s is listed multiple times in " + + "consumedOptions, potentially also with " + + "fallback/deprecated key.", + execNodeClass.getCanonicalName(), consumedOption)); + } else { + nodeConfigOptions.put(configOption, tableConfig.get(configOption)); + } + } + nodeConfigOptions.forEach(persistedConfig::set); + return persistedConfig; + } + /** Helper Pojo used as a tuple for the {@link #LOOKUP_MAP}. */ private static final class ExecNodeNameVersion { diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/RexLiteralUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/RexLiteralUtil.java new file mode 100644 index 0000000000000..b981c4f2e5e08 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/RexLiteralUtil.java @@ -0,0 +1,195 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.utils; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.data.binary.BinaryStringData; +import org.apache.flink.table.planner.calcite.FlinkTypeFactory; +import org.apache.flink.table.planner.codegen.CodeGenException; +import org.apache.flink.table.types.logical.DistinctType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; + +import org.apache.calcite.avatica.util.ByteString; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rex.RexLiteral; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.calcite.util.DateString; +import org.apache.calcite.util.NlsString; +import org.apache.calcite.util.TimeString; +import org.apache.calcite.util.TimestampString; + +import java.math.BigDecimal; +import java.time.ZoneOffset; + +import static org.apache.flink.table.planner.utils.TimestampStringUtils.toLocalDateTime; + +/** Utilities to work with {@link RexLiteral}. */ +@Internal +public class RexLiteralUtil { + + private RexLiteralUtil() {} + + /** See {@link #toFlinkInternalValue(Comparable, LogicalType)}. */ + public static Tuple2 toFlinkInternalValue(RexLiteral literal) { + LogicalType targetType = FlinkTypeFactory.toLogicalType(literal.getType()); + return Tuple2.of( + toFlinkInternalValue(literal.getValueAs(Comparable.class), targetType), targetType); + } + + /** + * Convert a value from Calcite's {@link Comparable} data structures to Flink internal data + * structures and also tries to be a bit flexible by accepting usual Java types such as String + * and boxed numerics. + * + *

        In case of symbol types, this function will return provided value, checking that it's an + * {@link Enum}. + * + *

        This function is essentially like {@link FlinkTypeFactory#toLogicalType(RelDataType)} but + * for values. + * + *

        Check {@link RexLiteral#valueMatchesType(Comparable, SqlTypeName, boolean)} for details on + * the {@link Comparable} data structures and {@link org.apache.flink.table.data.RowData} for + * details on Flink's internal data structures. + * + * @param value the value in Calcite's {@link Comparable} data structures + * @param valueType the type of the value + * @return the value in Flink's internal data structures + * @throws IllegalArgumentException in case the class of value does not match the expectations + * of valueType + */ + public static Object toFlinkInternalValue(Comparable value, LogicalType valueType) { + if (value == null) { + return null; + } + switch (valueType.getTypeRoot()) { + case CHAR: + case VARCHAR: + if (value instanceof NlsString) { + return BinaryStringData.fromString(((NlsString) value).getValue()); + } + if (value instanceof String) { + return BinaryStringData.fromString((String) value); + } + break; + case BOOLEAN: + if (value instanceof Boolean) { + return value; + } + break; + case BINARY: + case VARBINARY: + if (value instanceof ByteString) { + return ((ByteString) value).getBytes(); + } + break; + case DECIMAL: + if (value instanceof BigDecimal) { + return DecimalData.fromBigDecimal( + (BigDecimal) value, + LogicalTypeChecks.getPrecision(valueType), + LogicalTypeChecks.getScale(valueType)); + } + break; + case TINYINT: + if (value instanceof Number) { + return ((Number) value).byteValue(); + } + break; + case SMALLINT: + if (value instanceof Number) { + return ((Number) value).shortValue(); + } + break; + case INTEGER: + case INTERVAL_YEAR_MONTH: + if (value instanceof Number) { + return ((Number) value).intValue(); + } + break; + case BIGINT: + case INTERVAL_DAY_TIME: + if (value instanceof Number) { + return ((Number) value).longValue(); + } + break; + case FLOAT: + if (value instanceof Number) { + return ((Number) value).floatValue(); + } + break; + case DOUBLE: + if (value instanceof Number) { + return ((Number) value).doubleValue(); + } + break; + case DATE: + if (value instanceof DateString) { + return ((DateString) value).getDaysSinceEpoch(); + } + if (value instanceof Number) { + return ((Number) value).intValue(); + } + break; + case TIME_WITHOUT_TIME_ZONE: + if (value instanceof TimeString) { + return ((TimeString) value).getMillisOfDay(); + } + if (value instanceof Number) { + return ((Number) value).intValue(); + } + break; + case TIMESTAMP_WITHOUT_TIME_ZONE: + if (value instanceof TimestampString) { + return TimestampData.fromLocalDateTime( + toLocalDateTime((TimestampString) value)); + } + break; + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + if (value instanceof TimestampString) { + return TimestampData.fromInstant( + toLocalDateTime((TimestampString) value) + .atOffset(ZoneOffset.UTC) + .toInstant()); + } + break; + case DISTINCT_TYPE: + return toFlinkInternalValue(value, ((DistinctType) valueType).getSourceType()); + case SYMBOL: + if (value instanceof Enum) { + return value; + } + break; + case TIMESTAMP_WITH_TIME_ZONE: + case ARRAY: + case MULTISET: + case MAP: + case ROW: + case STRUCTURED_TYPE: + case NULL: + case UNRESOLVED: + throw new CodeGenException("Type not supported: " + valueType); + } + throw new IllegalStateException( + "Unexpected class " + value.getClass() + " for value of type " + valueType); + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/DummyStreamExecutionEnvironment.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/DummyStreamExecutionEnvironment.java index e3db5338d84c2..02faf81195f74 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/DummyStreamExecutionEnvironment.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/DummyStreamExecutionEnvironment.java @@ -24,6 +24,7 @@ import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.TimeCharacteristic; @@ -84,6 +85,11 @@ public ExecutionConfig getConfig() { return realExecEnv.getConfig(); } + @Override + public ReadableConfig getConfiguration() { + return realExecEnv.getConfiguration(); + } + @Override public List> getCachedFiles() { return realExecEnv.getCachedFiles(); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/ShortcutUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/ShortcutUtils.java index 90914da7c918d..7cc41f1220fec 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/ShortcutUtils.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/ShortcutUtils.java @@ -89,7 +89,7 @@ public static FlinkContext unwrapContext(Context context) { } public static ReadableConfig unwrapTableConfig(RelNode relNode) { - return unwrapContext(relNode).getTableConfig().getConfiguration(); + return unwrapContext(relNode).getTableConfig(); } public static @Nullable FunctionDefinition unwrapFunctionDefinition( diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/TableConfigUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/TableConfigUtils.java index 592dcf8a22e23..7c22ac0c6dea7 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/TableConfigUtils.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/TableConfigUtils.java @@ -18,14 +18,19 @@ package org.apache.flink.table.planner.utils; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.api.config.ExecutionConfigOptions; +import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.planner.calcite.CalciteConfig; import org.apache.flink.table.planner.calcite.CalciteConfig$; import org.apache.flink.table.planner.plan.utils.OperatorType; +import java.time.ZoneId; import java.util.HashSet; import java.util.Set; +import static java.time.ZoneId.SHORT_IDS; import static org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS; import static org.apache.flink.table.api.config.OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY; @@ -40,7 +45,7 @@ public class TableConfigUtils { * @return true if the given operator is disabled. */ public static boolean isOperatorDisabled(TableConfig tableConfig, OperatorType operatorType) { - String value = tableConfig.getConfiguration().getString(TABLE_EXEC_DISABLED_OPERATORS); + String value = tableConfig.get(TABLE_EXEC_DISABLED_OPERATORS); if (value == null) { return false; } @@ -68,8 +73,7 @@ public static boolean isOperatorDisabled(TableConfig tableConfig, OperatorType o * @return the aggregate phase strategy */ public static AggregatePhaseStrategy getAggPhaseStrategy(TableConfig tableConfig) { - String aggPhaseConf = - tableConfig.getConfiguration().getString(TABLE_OPTIMIZER_AGG_PHASE_STRATEGY).trim(); + String aggPhaseConf = tableConfig.get(TABLE_OPTIMIZER_AGG_PHASE_STRATEGY).trim(); if (aggPhaseConf.isEmpty()) { return AggregatePhaseStrategy.AUTO; } else { @@ -90,6 +94,45 @@ public static CalciteConfig getCalciteConfig(TableConfig tableConfig) { .orElse(CalciteConfig$.MODULE$.DEFAULT()); } + /** + * Similar to {@link TableConfig#getLocalTimeZone()} but extracting it from a generic {@link + * ReadableConfig}. + * + * @see TableConfig#getLocalTimeZone() + */ + public static ZoneId getLocalTimeZone(ReadableConfig tableConfig) { + String zone = tableConfig.get(TableConfigOptions.LOCAL_TIME_ZONE); + validateTimeZone(zone); + return TableConfigOptions.LOCAL_TIME_ZONE.defaultValue().equals(zone) + ? ZoneId.systemDefault() + : ZoneId.of(zone); + } + + /** + * Similar to {@link TableConfig#getMaxIdleStateRetentionTime()}. + * + * @see TableConfig#getMaxIdleStateRetentionTime() + */ + @Deprecated + public static long getMaxIdleStateRetentionTime(ReadableConfig tableConfig) { + return tableConfig.get(ExecutionConfigOptions.IDLE_STATE_RETENTION).toMillis() * 3 / 2; + } + + /** Validates user configured time zone. */ + private static void validateTimeZone(String zone) { + final String zoneId = zone.toUpperCase(); + if (zoneId.startsWith("UTC+") + || zoneId.startsWith("UTC-") + || SHORT_IDS.containsKey(zoneId)) { + throw new IllegalArgumentException( + String.format( + "The supported Zone ID is either a full name such as " + + "'America/Los_Angeles', or a custom timezone id such as " + + "'GMT-08:00', but configured Zone ID is '%s'.", + zone)); + } + } + // Make sure that we cannot instantiate this class private TableConfigUtils() {} } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelBuilder.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelBuilder.scala deleted file mode 100644 index b96e51023bc12..0000000000000 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelBuilder.scala +++ /dev/null @@ -1,233 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.planner.calcite - -import org.apache.flink.table.operations.QueryOperation -import org.apache.flink.table.planner.calcite.FlinkRelFactories.{ExpandFactory, RankFactory} -import org.apache.flink.table.planner.expressions.WindowProperty -import org.apache.flink.table.planner.plan.QueryOperationConverter -import org.apache.flink.table.planner.plan.logical.LogicalWindow -import org.apache.flink.table.planner.plan.nodes.calcite.{LogicalTableAggregate, LogicalWatermarkAssigner, LogicalWindowAggregate, LogicalWindowTableAggregate} -import org.apache.flink.table.planner.plan.utils.AggregateUtil -import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty -import org.apache.flink.table.runtime.operators.rank.{RankRange, RankType} - -import com.google.common.collect.ImmutableList -import org.apache.calcite.plan._ -import org.apache.calcite.rel.RelCollation -import org.apache.calcite.rel.`type`.RelDataTypeField -import org.apache.calcite.rel.hint.RelHint -import org.apache.calcite.rel.logical.LogicalAggregate -import org.apache.calcite.rex.RexNode -import org.apache.calcite.sql.SqlKind -import org.apache.calcite.tools.RelBuilder.{AggCall, Config, GroupKey} -import org.apache.calcite.tools.{RelBuilder, RelBuilderFactory} -import org.apache.calcite.util.{ImmutableBitSet, Util} -import org.apache.flink.table.catalog.ObjectIdentifier -import org.apache.flink.table.planner.hint.FlinkHints - -import java.lang.Iterable -import java.util -import java.util.List -import java.util.function.UnaryOperator - -import scala.collection.JavaConversions._ - -/** - * Flink specific [[RelBuilder]] that changes the default type factory to a [[FlinkTypeFactory]]. - */ -class FlinkRelBuilder( - context: Context, - relOptCluster: RelOptCluster, - relOptSchema: RelOptSchema) - extends RelBuilder( - context, - relOptCluster, - relOptSchema) { - - require(context != null) - - private val toRelNodeConverter = { - new QueryOperationConverter(this, context.unwrap(classOf[FlinkContext]).isBatchMode) - } - - private val expandFactory: ExpandFactory = { - Util.first(context.unwrap(classOf[ExpandFactory]), FlinkRelFactories.DEFAULT_EXPAND_FACTORY) - } - - private val rankFactory: RankFactory = { - Util.first(context.unwrap(classOf[RankFactory]), FlinkRelFactories.DEFAULT_RANK_FACTORY) - } - - override def getRelOptSchema: RelOptSchema = relOptSchema - - override def getCluster: RelOptCluster = relOptCluster - - override def getTypeFactory: FlinkTypeFactory = - super.getTypeFactory.asInstanceOf[FlinkTypeFactory] - - override def transform(transform: UnaryOperator[RelBuilder.Config]): FlinkRelBuilder = { - // Override in order to return a FlinkRelBuilder. - FlinkRelBuilder.of(transform.apply(Config.DEFAULT), cluster, relOptSchema) - } - - def expand( - projects: util.List[util.List[RexNode]], - expandIdIndex: Int): RelBuilder = { - val input = build() - val expand = expandFactory.createExpand(input, projects, expandIdIndex) - push(expand) - } - - def rank( - partitionKey: ImmutableBitSet, - orderKey: RelCollation, - rankType: RankType, - rankRange: RankRange, - rankNumberType: RelDataTypeField, - outputRankNumber: Boolean): RelBuilder = { - val input = build() - val rank = rankFactory.createRank(input, partitionKey, orderKey, rankType, rankRange, - rankNumberType, outputRankNumber) - push(rank) - } - - /** - * Build non-window aggregate for either aggregate or table aggregate. - */ - override def aggregate(groupKey: GroupKey, aggCalls: Iterable[AggCall]): RelBuilder = { - // build a relNode, the build() may also return a project - val relNode = super.aggregate(groupKey, aggCalls).build() - - def isCountStartAgg(agg: LogicalAggregate): Boolean = { - if (agg.getGroupCount != 0 || agg.getAggCallList.size() != 1) { - return false - } - val call = agg.getAggCallList.head - call.getAggregation.getKind == SqlKind.COUNT && - call.filterArg == -1 && call.getArgList.isEmpty - } - - relNode match { - case logicalAggregate: LogicalAggregate - if AggregateUtil.isTableAggregate(logicalAggregate.getAggCallList) => - push(LogicalTableAggregate.create(logicalAggregate)) - case logicalAggregate2: LogicalAggregate - if isCountStartAgg(logicalAggregate2) => - val newAggInput = push(logicalAggregate2.getInput(0)) - .project(literal(0)).build() - push(logicalAggregate2.copy(logicalAggregate2.getTraitSet, ImmutableList.of(newAggInput))) - case _ => push(relNode) - } - } - - /** - * Build window aggregate for either aggregate or table aggregate. - */ - def windowAggregate( - window: LogicalWindow, - groupKey: GroupKey, - namedProperties: List[NamedWindowProperty], - aggCalls: Iterable[AggCall]): RelBuilder = { - // build logical aggregate - - // Because of: - // [CALCITE-3763] RelBuilder.aggregate should prune unused fields from the input, - // if the input is a Project. - // - // the field can not be pruned if it is referenced by other expressions - // of the window aggregation(i.e. the TUMBLE_START/END). - // To solve this, we config the RelBuilder to forbidden this feature. - val aggregate = super.transform( - new UnaryOperator[RelBuilder.Config] { - override def apply(t: RelBuilder.Config) - : RelBuilder.Config = t.withPruneInputOfAggregate(false) - }) - .push(build()) - .aggregate(groupKey, aggCalls) - .build() - .asInstanceOf[LogicalAggregate] - - // build logical window aggregate from it - aggregate match { - case logicalAggregate: LogicalAggregate - if AggregateUtil.isTableAggregate(logicalAggregate.getAggCallList) => - push(LogicalWindowTableAggregate.create(window, namedProperties, aggregate)) - case _ => push(LogicalWindowAggregate.create(window, namedProperties, aggregate)) - } - } - - /** - * Build watermark assigner relation node. - */ - def watermark(rowtimeFieldIndex: Int, watermarkExpr: RexNode): RelBuilder = { - val input = build() - val watermarkAssigner = LogicalWatermarkAssigner - .create(cluster, input, rowtimeFieldIndex, watermarkExpr) - push(watermarkAssigner) - this - } - - def queryOperation(queryOperation: QueryOperation): RelBuilder = { - val relNode = queryOperation.accept(toRelNodeConverter) - push(relNode) - this - } - - def scan( - identifier: ObjectIdentifier, - dynamicOptions: util.Map[String, String]): RelBuilder = { - val hints = new util.ArrayList[RelHint] - hints.add(RelHint.builder(FlinkHints.HINT_NAME_OPTIONS).hintOptions(dynamicOptions).build) - val toRelContext = ViewExpanders.simpleContext(cluster, hints) - push(relOptSchema.getTableForMember(identifier.toList).toRel(toRelContext)) - this - } -} - -object FlinkRelBuilder { - - case class NamedWindowProperty(name: String, property: WindowProperty) - - def proto(context: Context): RelBuilderFactory = new RelBuilderFactory() { - def create(cluster: RelOptCluster, schema: RelOptSchema): RelBuilder = { - val clusterContext = cluster.getPlanner.getContext.unwrap(classOf[FlinkContext]) - val mergedContext = Contexts.chain(context, clusterContext) - - new FlinkRelBuilder(mergedContext, cluster, schema) - } - } - - def of(cluster: RelOptCluster, relOptSchema: RelOptSchema): FlinkRelBuilder = { - val clusterContext = cluster.getPlanner.getContext - new FlinkRelBuilder( - clusterContext, - cluster, - relOptSchema) - } - - def of(contextVar: Object, cluster: RelOptCluster, relOptSchema: RelOptSchema) - : FlinkRelBuilder = { - val mergedContext = Contexts.of(contextVar, cluster.getPlanner.getContext) - new FlinkRelBuilder( - mergedContext, - cluster, - relOptSchema) - } -} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/PreValidateReWriter.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/PreValidateReWriter.scala index 39c0aa5c000a1..3d930b0968b06 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/PreValidateReWriter.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/PreValidateReWriter.scala @@ -21,6 +21,7 @@ package org.apache.flink.table.planner.calcite import org.apache.flink.sql.parser.SqlProperty import org.apache.flink.sql.parser.dml.RichSqlInsert import org.apache.flink.sql.parser.dql.SqlRichExplain +import org.apache.flink.sql.parser.`type`.SqlMapTypeNameSpec import org.apache.flink.table.api.ValidationException import org.apache.flink.table.planner.calcite.PreValidateReWriter.{appendPartitionAndNullsProjects, notSupported} import org.apache.flink.table.planner.plan.schema.{CatalogSourceTable, FlinkPreparingTableBase, LegacyCatalogSourceTable} @@ -30,12 +31,12 @@ import org.apache.calcite.plan.RelOptTable import org.apache.calcite.prepare.CalciteCatalogReader import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory, RelDataTypeField} import org.apache.calcite.runtime.{CalciteContextException, Resources} -import org.apache.calcite.sql.`type`.SqlTypeUtil +import org.apache.calcite.sql.`type`.{SqlTypeName, SqlTypeUtil} import org.apache.calcite.sql.fun.SqlStdOperatorTable import org.apache.calcite.sql.parser.SqlParserPos import org.apache.calcite.sql.util.SqlBasicVisitor import org.apache.calcite.sql.validate.{SqlValidatorException, SqlValidatorTable, SqlValidatorUtil} -import org.apache.calcite.sql.{SqlCall, SqlIdentifier, SqlKind, SqlLiteral, SqlNode, SqlNodeList, SqlOrderBy, SqlSelect, SqlUtil} +import org.apache.calcite.sql.{SqlCall, SqlDataTypeSpec, SqlIdentifier, SqlKind, SqlLiteral, SqlNode, SqlNodeList, SqlOrderBy, SqlSelect, SqlUtil} import org.apache.calcite.util.Static.RESOURCE import java.util @@ -391,8 +392,21 @@ object PreValidateReWriter { == desiredType)) { node } else { - SqlStdOperatorTable.CAST.createCall(SqlParserPos.ZERO, - node, SqlTypeUtil.convertTypeToSpec(desiredType)) + // See FLINK-26460 for more details + val sqlDataTypeSpec = + if (SqlTypeUtil.isNull(currentType) && SqlTypeUtil.isMap(desiredType)) { + val keyType = desiredType.getKeyType + val valueType = desiredType.getValueType + new SqlDataTypeSpec( + new SqlMapTypeNameSpec( + SqlTypeUtil.convertTypeToSpec(keyType).withNullable(keyType.isNullable), + SqlTypeUtil.convertTypeToSpec(valueType).withNullable(valueType.isNullable), + SqlParserPos.ZERO), + SqlParserPos.ZERO) + } else { + SqlTypeUtil.convertTypeToSpec(desiredType) + } + SqlStdOperatorTable.CAST.createCall(SqlParserPos.ZERO, node, sqlDataTypeSpec) } } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala index a015377cdd154..683b23183f82a 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala @@ -209,13 +209,27 @@ object CodeGenUtils { case _: JShort => s"((short)$value)" case _: JInt => value.toString case _: JLong => value.toString + "L" - case _: JFloat => value.toString + "f" - case _: JDouble => value.toString + "d" + case _: JFloat => value match { + case JFloat.NEGATIVE_INFINITY => "java.lang.Float.NEGATIVE_INFINITY" + case JFloat.POSITIVE_INFINITY => "java.lang.Float.POSITIVE_INFINITY" + case _ => value.toString + "f" + } + case _: JDouble => value match { + case JDouble.NEGATIVE_INFINITY => "java.lang.Double.NEGATIVE_INFINITY" + case JDouble.POSITIVE_INFINITY => "java.lang.Double.POSITIVE_INFINITY" + case _ => value.toString + "d" + } case sd: StringData => qualifyMethod(BINARY_STRING_DATA_FROM_STRING) + "(\"" + EncodingUtils.escapeJava(sd.toString) + "\")" case td: TimestampData => s"$TIMESTAMP_DATA.fromEpochMillis(${td.getMillisecond}L, ${td.getNanoOfMillisecond})" + case decimalData: DecimalData => + s"""$DECIMAL_UTIL.castFrom( + |"${decimalData.toString}", + |${decimalData.precision()}, + |${decimalData.scale()})""" + .stripMargin case _ => throw new IllegalArgumentException("Illegal literal type: " + value.getClass) } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGeneratorContext.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGeneratorContext.scala index ab8ba8f3a7904..f55295808c44b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGeneratorContext.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGeneratorContext.scala @@ -420,14 +420,26 @@ class CodeGeneratorContext(val tableConfig: TableConfig) { } val addElementsCode = elements.map { element => - s""" - |${element.code} - |if (${element.nullTerm}) { - | $fieldTerm.addNull(); - |} else { - | $fieldTerm.add(${element.resultTerm}); - |} - |""".stripMargin + if (element.literalValue.isDefined) { + // Don't generate the null check in case the element is a literal expression + if (element.literalValue.get != null) { + s""" + |${element.code} + |$fieldTerm.add(${element.resultTerm}); + |""".stripMargin + } else if (element.literalValue.get == null) { + s"$fieldTerm.addNull();" + } + } else { + s""" + |${element.code} + |if (${element.nullTerm}) { + | $fieldTerm.addNull(); + |} else { + | $fieldTerm.add(${element.resultTerm}); + |} + |""".stripMargin + } }.mkString("\n") val setBuildingFunctionName = newName("buildSet") val setBuildingFunctionCode = @@ -886,8 +898,11 @@ class CodeGeneratorContext(val tableConfig: TableConfig) { /** * Adds a reusable string constant to the member area of the generated class. + * + * The string must be already escaped with + * [[org.apache.flink.table.utils.EncodingUtils.escapeJava()]]. */ - def addReusableStringConstants(value: String): String = { + def addReusableEscapedStringConstant(value: String): String = { reusableStringConstants.get(value) match { case Some(field) => field case None => diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/ExprCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/ExprCodeGenerator.scala index aebaf6b181d6f..3c3f8581bf625 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/ExprCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/ExprCodeGenerator.scala @@ -18,28 +18,25 @@ package org.apache.flink.table.planner.codegen -import org.apache.calcite.rex._ -import org.apache.calcite.sql.`type`.{ReturnTypes, SqlTypeName} -import org.apache.calcite.sql.{SqlKind, SqlOperator} -import org.apache.calcite.util.{Sarg, TimestampString} import org.apache.flink.streaming.api.functions.ProcessFunction import org.apache.flink.table.api.TableException import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.data.RowData import org.apache.flink.table.data.binary.BinaryRowData -import org.apache.flink.table.data.util.DataFormatConverters.{DataFormatConverter, getConverterForDataType} +import org.apache.flink.table.data.util.DataFormatConverters.{getConverterForDataType, DataFormatConverter} import org.apache.flink.table.functions.BuiltInFunctionDefinitions -import org.apache.flink.table.planner.calcite.{FlinkRexBuilder, FlinkTypeFactory, RexDistinctKeyVariable, RexFieldVariable} +import org.apache.flink.table.planner.calcite.{FlinkTypeFactory, RexDistinctKeyVariable, RexFieldVariable} import org.apache.flink.table.planner.codegen.CodeGenUtils._ import org.apache.flink.table.planner.codegen.GenerateUtils._ import org.apache.flink.table.planner.codegen.GeneratedExpression.{NEVER_NULL, NO_CODE} import org.apache.flink.table.planner.codegen.calls.ScalarOperatorGens._ +import org.apache.flink.table.planner.codegen.calls.SearchOperatorGen.generateSearch import org.apache.flink.table.planner.codegen.calls._ import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction import org.apache.flink.table.planner.functions.sql.FlinkSqlOperatorTable._ import org.apache.flink.table.planner.functions.sql.SqlThrowExceptionFunction import org.apache.flink.table.planner.functions.utils.{ScalarSqlFunction, TableSqlFunction} -import org.apache.flink.table.planner.plan.utils.FlinkRexUtil +import org.apache.flink.table.planner.plan.utils.RexLiteralUtil import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType import org.apache.flink.table.runtime.types.PlannerTypeUtils.isInteroperable import org.apache.flink.table.runtime.typeutils.TypeCheckUtils @@ -48,6 +45,10 @@ import org.apache.flink.table.types.logical._ import org.apache.flink.table.types.logical.utils.LogicalTypeChecks.{getFieldCount, isCompositeType} import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo +import org.apache.calcite.rex._ +import org.apache.calcite.sql.`type`.{ReturnTypes, SqlTypeName} +import org.apache.calcite.sql.{SqlKind, SqlOperator} + import scala.collection.JavaConversions._ /** @@ -411,15 +412,8 @@ class ExprCodeGenerator(ctx: CodeGeneratorContext, nullableInput: Boolean) } override def visitLiteral(literal: RexLiteral): GeneratedExpression = { - val resultType = FlinkTypeFactory.toLogicalType(literal.getType) - val value = resultType.getTypeRoot match { - case LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE | - LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE => - literal.getValueAs(classOf[TimestampString]) - case _ => - literal.getValue3 - } - generateLiteral(ctx, resultType, value) + val res = RexLiteralUtil.toFlinkInternalValue(literal) + generateLiteral(ctx, res.f0, res.f1) } override def visitCorrelVariable(correlVariable: RexCorrelVariable): GeneratedExpression = { @@ -471,19 +465,10 @@ class ExprCodeGenerator(ctx: CodeGeneratorContext, nullableInput: Boolean) override def visitCall(call: RexCall): GeneratedExpression = { val resultType = FlinkTypeFactory.toLogicalType(call.getType) if (call.getKind == SqlKind.SEARCH) { - val sarg = call.getOperands.get(1).asInstanceOf[RexLiteral] - .getValueAs(classOf[Sarg[_]]) - val rexBuilder = new FlinkRexBuilder(FlinkTypeFactory.INSTANCE) - if (sarg.isPoints) { - val operands = FlinkRexUtil.expandSearchOperands(rexBuilder, call) - .map(operand => operand.accept(this)) - return generateCallExpression(ctx, call, operands, resultType) - } else { - return RexUtil.expandSearch( - rexBuilder, - null, - call).accept(this) - } + return generateSearch( + ctx, + generateExpression(call.getOperands.get(0)), + call.getOperands.get(1).asInstanceOf[RexLiteral]) } // convert operands and help giving untyped NULL literals a type @@ -691,16 +676,6 @@ class ExprCodeGenerator(ctx: CodeGeneratorContext, nullableInput: Boolean) requireBoolean(operand) generateIsNotFalse(operand) - case SEARCH | IN => - val left = operands.head - val right = operands.tail - generateIn(ctx, left, right) - - case NOT_IN => - val left = operands.head - val right = operands.tail - generateNot(ctx, generateIn(ctx, left, right)) - // casting case CAST => generateCast(ctx, operands.head, resultType, nullOnFailure = ctx.tableConfig @@ -876,8 +851,8 @@ class ExprCodeGenerator(ctx: CodeGeneratorContext, nullableInput: Boolean) case None => null case Some(literal) => getConverterForDataType(fromLogicalTypeToDataType(expr.resultType)) - .asInstanceOf[DataFormatConverter[AnyRef, AnyRef] - ].toExternal(literal.asInstanceOf[AnyRef]) + .asInstanceOf[DataFormatConverter[AnyRef, AnyRef]] + .toExternal(literal.asInstanceOf[AnyRef]) } }.toArray } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala index be0f4532aff38..5c6018d8d8f79 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala @@ -21,7 +21,7 @@ package org.apache.flink.table.planner.codegen import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeinfo.{AtomicType => AtomicTypeInfo} import org.apache.flink.table.data._ -import org.apache.flink.table.data.binary.BinaryRowData +import org.apache.flink.table.data.binary.{BinaryRowData, BinaryStringData} import org.apache.flink.table.data.utils.JoinedRowData import org.apache.flink.table.data.writer.BinaryRowWriter import org.apache.flink.table.planner.codegen.CodeGenUtils._ @@ -30,18 +30,11 @@ import org.apache.flink.table.planner.codegen.calls.CurrentTimePointCallGen import org.apache.flink.table.planner.plan.nodes.exec.spec.SortSpec import org.apache.flink.table.planner.plan.utils.SortUtil import org.apache.flink.table.planner.typeutils.SymbolUtil.calciteToCommon -import org.apache.flink.table.planner.utils.TimestampStringUtils.toLocalDateTime import org.apache.flink.table.runtime.typeutils.TypeCheckUtils.{isCharacterString, isReference, isTemporal} -import org.apache.flink.table.types.logical.LogicalTypeRoot._ import org.apache.flink.table.types.logical._ +import org.apache.flink.table.types.logical.LogicalTypeRoot._ import org.apache.flink.table.types.logical.utils.LogicalTypeChecks.{getFieldCount, getFieldTypes} - -import org.apache.calcite.avatica.util.ByteString -import org.apache.calcite.util.TimestampString -import org.apache.commons.lang3.StringEscapeUtils - -import java.math.{BigDecimal => JBigDecimal} -import java.time.ZoneOffset +import org.apache.flink.table.utils.EncodingUtils import scala.annotation.tailrec import scala.collection.mutable @@ -313,164 +306,53 @@ object GenerateUtils { literalValue = Some(literalValue)) } + /** + * This function accepts the Flink's internal data structures. + * + * Check [[org.apache.flink.table.planner.plan.utils.RexLiteralUtil.toFlinkInternalValue]] to + * convert RexLiteral value to Flink's internal data structures. + */ @tailrec def generateLiteral( ctx: CodeGeneratorContext, - literalType: LogicalType, - literalValue: Any): GeneratedExpression = { + literalValue: Any, + literalType: LogicalType): GeneratedExpression = { if (literalValue == null) { return generateNullLiteral(literalType, ctx.nullCheck) } - // non-null values literalType.getTypeRoot match { - // ordered by type root definition + // For strings, binary and decimal, we add the literal as reusable field, + // as they're not cheap to construct. For the other types, the return term is directly + // the literal value case CHAR | VARCHAR => - val escapedValue = StringEscapeUtils.ESCAPE_JAVA.translate(literalValue.toString) - val field = ctx.addReusableStringConstants(escapedValue) - generateNonNullLiteral(literalType, field, StringData.fromString(escapedValue)) - - case BOOLEAN => - generateNonNullLiteral(literalType, literalValue.toString, literalValue) + val str = literalValue.asInstanceOf[BinaryStringData] + val field = ctx.addReusableEscapedStringConstant(EncodingUtils.escapeJava(str.toString)) + generateNonNullLiteral(literalType, field, str) case BINARY | VARBINARY => - val bytesVal = literalValue.asInstanceOf[ByteString].getBytes + val bytesVal = literalValue.asInstanceOf[Array[Byte]] val fieldTerm = ctx.addReusableObject( bytesVal, "binary", bytesVal.getClass.getCanonicalName) generateNonNullLiteral(literalType, fieldTerm, bytesVal) case DECIMAL => - val dt = literalType.asInstanceOf[DecimalType] - val precision = dt.getPrecision - val scale = dt.getScale val fieldTerm = newName("decimal") - val decimalClass = className[DecimalData] - val fieldDecimal = - s""" - |$decimalClass $fieldTerm = - | $DECIMAL_UTIL.castFrom("${literalValue.toString}", $precision, $scale); - |""".stripMargin - ctx.addReusableMember(fieldDecimal) - val value = DecimalData.fromBigDecimal( - literalValue.asInstanceOf[JBigDecimal], precision, scale) - if (value == null) { - generateNullLiteral(literalType, ctx.nullCheck) - } else { - generateNonNullLiteral(literalType, fieldTerm, value) - } - - case TINYINT => - val decimal = BigDecimal(literalValue.asInstanceOf[JBigDecimal]) - generateNonNullLiteral(literalType, decimal.byteValue().toString, decimal.byteValue()) - - case SMALLINT => - val decimal = BigDecimal(literalValue.asInstanceOf[JBigDecimal]) - generateNonNullLiteral(literalType, decimal.shortValue().toString, decimal.shortValue()) - - case INTEGER => - val decimal = BigDecimal(literalValue.asInstanceOf[JBigDecimal]) - generateNonNullLiteral(literalType, decimal.intValue().toString, decimal.intValue()) - - case BIGINT => - val decimal = BigDecimal(literalValue.asInstanceOf[JBigDecimal]) - generateNonNullLiteral( - literalType, decimal.longValue().toString + "L", decimal.longValue()) - - case FLOAT => - val floatValue = literalValue.asInstanceOf[JBigDecimal].floatValue() - floatValue match { - case Float.NegativeInfinity => - generateNonNullLiteral( - literalType, - "java.lang.Float.NEGATIVE_INFINITY", - Float.NegativeInfinity) - case Float.PositiveInfinity => generateNonNullLiteral( - literalType, - "java.lang.Float.POSITIVE_INFINITY", - Float.PositiveInfinity) - case _ => generateNonNullLiteral( - literalType, floatValue.toString + "f", floatValue) - } - - case DOUBLE => - val doubleValue = literalValue.asInstanceOf[JBigDecimal].doubleValue() - doubleValue match { - case Double.NegativeInfinity => - generateNonNullLiteral( - literalType, - "java.lang.Double.NEGATIVE_INFINITY", - Double.NegativeInfinity) - case Double.PositiveInfinity => - generateNonNullLiteral( - literalType, - "java.lang.Double.POSITIVE_INFINITY", - Double.PositiveInfinity) - case _ => generateNonNullLiteral( - literalType, doubleValue.toString + "d", doubleValue) - } - - case DATE => - generateNonNullLiteral(literalType, literalValue.toString, literalValue) - - case TIME_WITHOUT_TIME_ZONE => - generateNonNullLiteral(literalType, literalValue.toString, literalValue) - - case TIMESTAMP_WITHOUT_TIME_ZONE => - val fieldTerm = newName("timestamp") - val ldt = toLocalDateTime(literalValue.asInstanceOf[TimestampString]) - val ts = TimestampData.fromLocalDateTime(ldt) - val fieldTimestamp = - s""" - |$TIMESTAMP_DATA $fieldTerm = - | $TIMESTAMP_DATA.fromEpochMillis(${ts.getMillisecond}L, ${ts.getNanoOfMillisecond}); - """.stripMargin - ctx.addReusableMember(fieldTimestamp) - generateNonNullLiteral(literalType, fieldTerm, ts) - - case TIMESTAMP_WITH_TIME_ZONE => - throw new UnsupportedOperationException("Unsupported type: " + literalType) - - case TIMESTAMP_WITH_LOCAL_TIME_ZONE => - val fieldTerm = newName("timestampWithLocalZone") - val ins = - toLocalDateTime(literalValue.asInstanceOf[TimestampString]) - .atOffset(ZoneOffset.UTC) - .toInstant - val ts = TimestampData.fromInstant(ins) - val fieldTimestampWithLocalZone = - s""" - |$TIMESTAMP_DATA $fieldTerm = - | $TIMESTAMP_DATA.fromEpochMillis(${ts.getMillisecond}L, ${ts.getNanoOfMillisecond}); - """.stripMargin - ctx.addReusableMember(fieldTimestampWithLocalZone) + ctx.addReusableMember(s""" + |${className[DecimalData]} $fieldTerm = ${primitiveLiteralForType(literalValue)}; + |""".stripMargin) generateNonNullLiteral(literalType, fieldTerm, literalValue) - case INTERVAL_YEAR_MONTH => - val decimal = BigDecimal(literalValue.asInstanceOf[JBigDecimal]) - if (decimal.isValidInt) { - generateNonNullLiteral(literalType, decimal.intValue().toString, decimal.intValue()) - } else { - throw new CodeGenException( - s"Decimal '$decimal' can not be converted to interval of months.") - } - - case INTERVAL_DAY_TIME => - val decimal = BigDecimal(literalValue.asInstanceOf[JBigDecimal]) - if (decimal.isValidLong) { - generateNonNullLiteral( - literalType, - decimal.longValue().toString + "L", - decimal.longValue()) - } else { - throw new CodeGenException( - s"Decimal '$decimal' can not be converted to interval of milliseconds.") - } - case DISTINCT_TYPE => - generateLiteral(ctx, literalType.asInstanceOf[DistinctType].getSourceType, literalValue) + generateLiteral(ctx, literalValue, literalType.asInstanceOf[DistinctType].getSourceType) case SYMBOL => generateSymbol(literalValue.asInstanceOf[Enum[_]]) + case BOOLEAN | TINYINT | SMALLINT | INTEGER | BIGINT | FLOAT | DOUBLE | DATE | + TIME_WITHOUT_TIME_ZONE | TIMESTAMP_WITHOUT_TIME_ZONE | TIMESTAMP_WITH_LOCAL_TIME_ZONE | + INTERVAL_YEAR_MONTH | INTERVAL_DAY_TIME => + generateNonNullLiteral(literalType, primitiveLiteralForType(literalValue), literalValue) + case ARRAY | MULTISET | MAP | ROW | STRUCTURED_TYPE | NULL | UNRESOLVED => throw new CodeGenException(s"Type not supported: $literalType") } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/LookupJoinCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/LookupJoinCodeGenerator.scala index ef7b5b097edcb..5268a4b65855f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/LookupJoinCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/LookupJoinCodeGenerator.scala @@ -35,6 +35,7 @@ import org.apache.flink.table.planner.codegen.calls.BridgingFunctionGenUtil.veri import org.apache.flink.table.planner.delegation.PlannerBase import org.apache.flink.table.planner.functions.inference.LookupCallContext import org.apache.flink.table.planner.plan.utils.LookupJoinUtil.{ConstantLookupKey, FieldRefLookupKey, LookupKey} +import org.apache.flink.table.planner.plan.utils.RexLiteralUtil import org.apache.flink.table.planner.utils.JavaScalaConversionUtil.toScala import org.apache.flink.table.runtime.collector.{TableFunctionCollector, TableFunctionResultFuture} import org.apache.flink.table.runtime.generated.{GeneratedCollector, GeneratedFunction, GeneratedResultFuture} @@ -213,10 +214,8 @@ object LookupJoinCodeGenerator { .map(lookupKeys.get) .map { case constantKey: ConstantLookupKey => - generateLiteral( - ctx, - constantKey.sourceType, - constantKey.literal.getValue3) + val res = RexLiteralUtil.toFlinkInternalValue(constantKey.literal) + generateLiteral(ctx, res.f0, res.f1) case fieldKey: FieldRefLookupKey => generateInputAccess( ctx, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/LikeCallGen.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/LikeCallGen.scala index cd8035d22f57b..15bdacb8e02a9 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/LikeCallGen.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/LikeCallGen.scala @@ -109,16 +109,16 @@ class LikeCallGen extends CallGenerator { val middleMatcher = MIDDLE_PATTERN.matcher(newPattern) if (noneMatcher.matches()) { - val reusePattern = ctx.addReusableStringConstants(newPattern) + val reusePattern = ctx.addReusableEscapedStringConstant(newPattern) s"${terms.head}.equals($reusePattern)" } else if (beginMatcher.matches()) { - val field = ctx.addReusableStringConstants(beginMatcher.group(1)) + val field = ctx.addReusableEscapedStringConstant(beginMatcher.group(1)) s"${terms.head}.startsWith($field)" } else if (endMatcher.matches()) { - val field = ctx.addReusableStringConstants(endMatcher.group(1)) + val field = ctx.addReusableEscapedStringConstant(endMatcher.group(1)) s"${terms.head}.endsWith($field)" } else if (middleMatcher.matches()) { - val field = ctx.addReusableStringConstants(middleMatcher.group(1)) + val field = ctx.addReusableEscapedStringConstant(middleMatcher.group(1)) s"${terms.head}.contains($field)" } else { val field = className[SqlLikeChainChecker] diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala index 7dcb13c7813b9..543cc92d22036 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala @@ -334,82 +334,6 @@ object ScalarOperatorGens { // scalar expression generate utils // ---------------------------------------------------------------------------------------- - /** - * Generates IN expression using a HashSet - */ - def generateIn( - ctx: CodeGeneratorContext, - needle: GeneratedExpression, - haystack: Seq[GeneratedExpression]) - : GeneratedExpression = { - - // add elements to hash set if they are constant - if (haystack.forall(_.literal)) { - - // determine common numeric type - val widerType = toScala(findCommonType(asList(needle.resultType, haystack.head.resultType))) - .orElse(throw new CodeGenException(s"Unable to find common type of $needle and $haystack.")) - - // we need to normalize the values for the hash set - val castNumeric = widerType match { - case Some(t) => (value: GeneratedExpression) => - numericCasting(ctx, value.resultType, t)(value.resultTerm) - case None => (value: GeneratedExpression) => value.resultTerm - } - - val resultType = widerType match { - case Some(t) => t - case None => needle.resultType - } - - val elements = haystack.map { element => - element.copy( - castNumeric(element), // cast element to wider type - element.nullTerm, - element.code, - resultType) - } - val setTerm = ctx.addReusableHashSet(elements, resultType) - - val castedNeedle = needle.copy( - castNumeric(needle), // cast needle to wider type - needle.nullTerm, - needle.code, - resultType) - - val Seq(resultTerm, nullTerm) = newNames("result", "isNull") - val resultTypeTerm = primitiveTypeTermForType(new BooleanType()) - val defaultValue = primitiveDefaultValue(new BooleanType()) - - val operatorCode = if (ctx.nullCheck) { - s""" - |${castedNeedle.code} - |$resultTypeTerm $resultTerm = $defaultValue; - |boolean $nullTerm = true; - |if (!${castedNeedle.nullTerm}) { - | $resultTerm = $setTerm.contains(${castedNeedle.resultTerm}); - | $nullTerm = !$resultTerm && $setTerm.containsNull(); - |} - |""".stripMargin.trim - } - else { - s""" - |${castedNeedle.code} - |$resultTypeTerm $resultTerm = $setTerm.contains(${castedNeedle.resultTerm}); - |""".stripMargin.trim - } - - GeneratedExpression(resultTerm, nullTerm, operatorCode, new BooleanType()) - } else { - // we use a chain of ORs for a set that contains non-constant elements - haystack - .map(generateEquals(ctx, needle, _)) - .reduce((left, right) => - generateOr(ctx, left, right) - ) - } - } - /** * check the validity of implicit type conversion * See: https://cwiki.apache.org/confluence/display/FLINK/FLIP-154%3A+SQL+Implicit+Type+Coercion @@ -1457,10 +1381,11 @@ object ScalarOperatorGens { elements: Seq[GeneratedExpression], greatest: Boolean = true) : GeneratedExpression = { - val Seq(result, cur, nullTerm) = newNames("result", "cur", "nullTerm") + val Seq(result, tmpResult, cur, nullTerm) = newNames("result", "tmpResult", "cur", "nullTerm") val widerType = toScala(findCommonType(elements.map(element => element.resultType))) .orElse(throw new CodeGenException(s"Unable to find common type for $elements.")) - val resultTypeTerm = boxedTypeTermForType(widerType.get) + val boxedResultTypeTerm = boxedTypeTermForType(widerType.get) + val primitiveResultTypeTerm = primitiveTypeTermForType(widerType.get) def castIfNumeric(t: GeneratedExpression): String = { if (isNumeric(widerType.get)) { @@ -1474,13 +1399,13 @@ object ScalarOperatorGens { s""" | ${element.code} | if (!$nullTerm) { - | $resultTypeTerm $cur = ${castIfNumeric(element)}; + | $boxedResultTypeTerm $cur = ${castIfNumeric(element)}; | if (${element.nullTerm}) { | $nullTerm = true; | } else { - | int compareResult = $result.compareTo($cur); + | int compareResult = $tmpResult.compareTo($cur); | if (($greatest && compareResult < 0) || (compareResult > 0 && !$greatest)) { - | $result = $cur; + | $tmpResult = $cur; | } | } | } @@ -1489,11 +1414,12 @@ object ScalarOperatorGens { val code = s""" - | $resultTypeTerm $result = ${castIfNumeric(elements.head)}; + | $boxedResultTypeTerm $tmpResult = ${castIfNumeric(elements.head)}; + | $primitiveResultTypeTerm $result = ${primitiveDefaultValue(widerType.get)}; | boolean $nullTerm = false; | $elementsCode - | if ($nullTerm) { - | $result = null; + | if (!$nullTerm) { + | $result = $tmpResult; | } """.stripMargin GeneratedExpression(result, nullTerm, code, resultType) @@ -1903,6 +1829,7 @@ object ScalarOperatorGens { def toCodegenCastContext(ctx: CodeGeneratorContext): CodeGeneratorCastRule.Context = { new CodeGeneratorCastRule.Context { + override def isPrinting(): Boolean = false override def legacyBehaviour(): Boolean = isLegacyCastBehaviourEnabled(ctx) override def getSessionTimeZoneTerm: String = ctx.addReusableSessionTimeZone() override def declareVariable(ty: String, variablePrefix: String): String = @@ -1918,6 +1845,8 @@ object ScalarOperatorGens { def toCastContext(ctx: CodeGeneratorContext): CastRule.Context = { new CastRule.Context { + override def isPrinting(): Boolean = false + override def legacyBehaviour(): Boolean = isLegacyCastBehaviourEnabled(ctx) override def getSessionZoneId: ZoneId = ctx.tableConfig.getLocalTimeZone diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/SearchOperatorGen.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/SearchOperatorGen.scala new file mode 100644 index 0000000000000..42c68d267e260 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/SearchOperatorGen.scala @@ -0,0 +1,241 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.codegen.calls + +import org.apache.flink.table.planner.calcite.FlinkTypeFactory +import org.apache.flink.table.planner.codegen.{CodeGeneratorContext, CodeGenException, GeneratedExpression} +import org.apache.flink.table.planner.codegen.CodeGenUtils.newNames +import org.apache.flink.table.planner.codegen.GenerateUtils.{generateLiteral, generateNullLiteral} +import org.apache.flink.table.planner.codegen.calls.ScalarOperatorGens._ +import org.apache.flink.table.planner.functions.casting.CastRuleProvider +import org.apache.flink.table.planner.plan.utils.RexLiteralUtil.toFlinkInternalValue +import org.apache.flink.table.types.logical.{BooleanType, LogicalType} +import org.apache.flink.table.types.logical.utils.LogicalTypeMerging.findCommonType + +import org.apache.calcite.rex.RexLiteral +import org.apache.calcite.util.{RangeSets, Sarg} + +import java.util.Arrays.asList + +import scala.collection.JavaConverters._ + +/** + * Class containing utilities to implement the SEARCH operator. + * + * This does not implement [[CallGenerator]] as the interface does not fit, because the [[Sarg]] + * argument cannot be converted directly to [[GeneratedExpression]]. + */ +object SearchOperatorGen { + + /** + * Generates SEARCH expression using either an HashSet or a concatenation of OR, + * depending on whether the elements of the haystack are all literals or not. + * + * Note that both IN/NOT IN are converted to SEARCH when the set has only constant values, + * otherwise the IN/NOT IN are converted to a set of disjunctions. See + * [[org.apache.calcite.rex.RexBuilder#makeIn(org.apache.calcite.rex.RexNode, java.util.List)]]. + */ + def generateSearch( + ctx: CodeGeneratorContext, + target: GeneratedExpression, + sargLiteral: RexLiteral): GeneratedExpression = { + val sarg: Sarg[Nothing] = sargLiteral.getValueAs(classOf[Sarg[Nothing]]) + val targetType = target.resultType + val sargType = FlinkTypeFactory.toLogicalType(sargLiteral.getType) + + val commonType: LogicalType = findCommonType(asList(targetType, sargType)) + .orElseThrow(() => + new CodeGenException(s"Unable to find common type of $target and $sargLiteral.")) + + val needle = generateCast( + ctx, + target, + commonType, + nullOnFailure = false + ) + + // In case the search is among points we use the hashset implementation + if (sarg.isPoints || sarg.isComplementedPoints) { + val rangeSet = if (sarg.isPoints) sarg.rangeSet else sarg.rangeSet.complement() + val haystack = rangeSet + .asRanges() + .asScala + // We need to go through the generateLiteral to normalize the value from calcite + .map(r => toFlinkInternalValue(r.lowerEndpoint, sargType)) + // The elements are constant, we perform the cast immediately + .map(CastRuleProvider.cast(toCastContext(ctx), sargType, commonType, _)) + .map(generateLiteral(ctx, _, commonType)) + if (sarg.containsNull) { + haystack += generateNullLiteral(commonType, ctx.nullCheck) + } + val setTerm = ctx.addReusableHashSet(haystack.toSeq, commonType) + val negation = if (sarg.isComplementedPoints) "!" else "" + + val Seq(resultTerm, nullTerm) = newNames("result", "isNull") + + val operatorCode = if (ctx.nullCheck) { + s""" + |${needle.code} + |// --- Begin SEARCH ${target.resultTerm} + |boolean $resultTerm = false; + |boolean $nullTerm = true; + |if (!${needle.nullTerm}) { + | $resultTerm = $negation$setTerm.contains(${needle.resultTerm}); + | $nullTerm = !$resultTerm && $setTerm.containsNull(); + |} + |// --- End SEARCH ${target.resultTerm} + |""".stripMargin.trim + } + else { + s""" + |${needle.code} + |// --- Begin SEARCH ${target.resultTerm} + |boolean $resultTerm = $negation$setTerm.contains(${needle.resultTerm}); + |// --- End SEARCH ${target.resultTerm} + |""".stripMargin.trim + } + + GeneratedExpression(resultTerm, nullTerm, operatorCode, new BooleanType()) + } else { + // We copy the target to don't re-evaluate on each range check + val dummyTarget = target.copy(code = "") + + val rangeToExpression = new RangeToExpression(ctx, sargType, dummyTarget) + + // We use a chain of ORs and range comparisons + var rangeChecks: Seq[GeneratedExpression] = sarg + .rangeSet + .asRanges + .asScala + .toSeq + .map(RangeSets.map(_, rangeToExpression)) + + if (sarg.containsNull) { + rangeChecks = Seq(generateIsNull(ctx, target)) ++ rangeChecks + } + + val generatedRangeChecks = rangeChecks + .reduce((left, right) => generateOr(ctx, left, right)) + + // Add the target expression code + val finalCode = + s""" + |${target.code} + |// --- Begin SEARCH ${target.resultTerm} + |${generatedRangeChecks.code} + |// --- End SEARCH ${target.resultTerm} + |""".stripMargin.trim + generatedRangeChecks.copy(code = finalCode) + } + } + + private class RangeToExpression[C <: Comparable[C]]( + ctx: CodeGeneratorContext, + boundType: LogicalType, + target: GeneratedExpression) extends RangeSets.Handler[C, GeneratedExpression] { + + override def all(): GeneratedExpression = { + generateLiteral(ctx, true, new BooleanType()) + } + + /** + * lower <= target + */ + override def atLeast(lower: C): GeneratedExpression = { + generateComparison(ctx, "<=", lit(lower), target) + } + + /** + * target <= upper + */ + override def atMost(upper: C): GeneratedExpression = { + generateComparison(ctx, "<=", target, lit(upper)) + } + + /** + * lower < target + */ + override def greaterThan(lower: C): GeneratedExpression = { + generateComparison(ctx, "<", lit(lower), target) + } + + /** + * target < upper + */ + override def lessThan(upper: C): GeneratedExpression = { + generateComparison(ctx, "<", target, lit(upper)) + } + + /** + * value == target + */ + override def singleton(value: C): GeneratedExpression = { + generateComparison(ctx, "==", lit(value), target) + } + + /** + * lower <= target && target <= upper + */ + override def closed(lower: C, upper: C): GeneratedExpression = { + generateAnd( + ctx, + generateComparison(ctx, "<=", lit(lower), target), + generateComparison(ctx, "<=", target, lit(upper)) + ) + } + + /** + * lower <= target && target < upper + */ + override def closedOpen(lower: C, upper: C): GeneratedExpression = { + generateAnd( + ctx, + generateComparison(ctx, "<=", lit(lower), target), + generateComparison(ctx, "<", target, lit(upper)) + ) + } + + /** + * lower < target && target <= upper + */ + override def openClosed(lower: C, upper: C): GeneratedExpression = { + generateAnd( + ctx, + generateComparison(ctx, "<", lit(lower), target), + generateComparison(ctx, "<=", target, lit(upper)) + ) + } + + /** + * lower < target && target < upper + */ + override def open(lower: C, upper: C): GeneratedExpression = { + generateAnd( + ctx, + generateComparison(ctx, "<", lit(lower), target), + generateComparison(ctx, "<", target, lit(upper)) + ) + } + + private def lit(value: C): GeneratedExpression = { + generateLiteral(ctx, toFlinkInternalValue(value, boundType), boundType) + } + } + +} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/BatchPlanner.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/BatchPlanner.scala index efc4bb025f54b..10e37d3d06b4f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/BatchPlanner.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/BatchPlanner.scala @@ -67,8 +67,7 @@ class BatchPlanner( // deadlock breakup processors.add(new DeadlockBreakupProcessor()) // multiple input creation - if (getTableConfig.getConfiguration.getBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_MULTIPLE_INPUT_ENABLED)) { + if (getTableConfig.get(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTIPLE_INPUT_ENABLED)) { processors.add(new MultipleInputNodeCreationProcessor(false)) } processors.add(new ForwardHashExchangeProcessor) @@ -154,7 +153,7 @@ class BatchPlanner( override def beforeTranslation(): Unit = { super.beforeTranslation() - val runtimeMode = getConfiguration.get(ExecutionOptions.RUNTIME_MODE) + val runtimeMode = getTableConfig.get(ExecutionOptions.RUNTIME_MODE) if (runtimeMode != RuntimeExecutionMode.BATCH) { throw new IllegalArgumentException( "Mismatch between configured runtime mode and actual runtime mode. " + diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala index bf6871c361a9a..599429284576e 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala @@ -18,18 +18,23 @@ package org.apache.flink.table.planner.delegation +import org.apache.calcite.jdbc.CalciteSchemaBuilder.asRootSchema +import org.apache.calcite.plan.{RelTrait, RelTraitDef} +import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.hint.RelHint +import org.apache.calcite.rel.logical.LogicalTableModify +import org.apache.calcite.tools.FrameworkConfig import org.apache.flink.annotation.VisibleForTesting import org.apache.flink.api.dag.Transformation import org.apache.flink.configuration.ReadableConfig import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import org.apache.flink.streaming.api.graph.StreamGraph import org.apache.flink.table.api._ -import org.apache.flink.table.api.config.{ExecutionConfigOptions, TableConfigOptions} +import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.catalog.ManagedTableListener.isManagedTable import org.apache.flink.table.catalog._ import org.apache.flink.table.connector.sink.DynamicTableSink import org.apache.flink.table.delegation.{Executor, Parser, Planner} -import org.apache.flink.table.descriptors.{ConnectorDescriptorValidator, DescriptorProperties} import org.apache.flink.table.factories.{DynamicTableSinkFactory, FactoryUtil, TableFactoryUtil} import org.apache.flink.table.module.{Module, ModuleManager} import org.apache.flink.table.operations.OutputConversionModifyOperation.UpdateMode @@ -59,17 +64,9 @@ import org.apache.flink.table.runtime.generated.CompileUtils import org.apache.flink.table.sinks.TableSink import org.apache.flink.table.types.utils.LegacyTypeInfoDataTypeConverter -import org.apache.calcite.jdbc.CalciteSchemaBuilder.asRootSchema -import org.apache.calcite.plan.{RelTrait, RelTraitDef} -import org.apache.calcite.rel.RelNode -import org.apache.calcite.rel.hint.RelHint -import org.apache.calcite.rel.logical.LogicalTableModify -import org.apache.calcite.tools.FrameworkConfig - import java.lang.{Long => JLong} import java.util import java.util.{Collections, TimeZone} - import _root_.scala.collection.JavaConversions._ import scala.collection.mutable @@ -103,10 +100,6 @@ abstract class PlannerBase( private var parser: Parser = _ private var currentDialect: SqlDialect = getTableConfig.getSqlDialect - private val plannerConfiguration: ReadableConfig = new PlannerConfig( - tableConfig.getConfiguration, - executor.getConfiguration) - @VisibleForTesting private[flink] val plannerContext: PlannerContext = new PlannerContext( @@ -148,14 +141,7 @@ abstract class PlannerBase( def getFlinkContext: FlinkContext = plannerContext.getFlinkContext /** - * Gives access to both API specific table configuration and executor configuration. - * - * This configuration should be the main source of truth in the planner module. - */ - def getConfiguration: ReadableConfig = plannerConfiguration - - /** - * @deprecated Do not use this method anymore. Use [[getConfiguration]] to access options. + * @deprecated Do not use this method anymore. Use [[getTableConfig]] to access options. * Create transformations without it. A [[StreamExecutionEnvironment]] is a mixture * of executor and stream graph generator/builder. In the long term, we would like * to avoid the need for it in the planner module. @@ -226,7 +212,7 @@ abstract class PlannerBase( getRelBuilder, input, collectModifyOperation, - getTableConfig.getConfiguration, + getTableConfig, getFlinkContext.getClassLoader ) @@ -397,12 +383,18 @@ abstract class PlannerBase( } if (!contextResolvedTable.isAnonymous && - isLegacyConnectorOptions(objectIdentifier, resolvedTable.getOrigin, isTemporary)) { + TableFactoryUtil.isLegacyConnectorOptions( + catalogManager.getCatalog(objectIdentifier.getCatalogName).orElse(null), + tableConfig, + isStreamingMode, + objectIdentifier, + resolvedTable.getOrigin, + isTemporary)) { val tableSink = TableFactoryUtil.findAndCreateTableSink( catalog.orNull, objectIdentifier, tableToFind.getOrigin, - getTableConfig.getConfiguration, + getTableConfig, isStreamingMode, isTemporary) Option(resolvedTable, tableSink) @@ -424,7 +416,7 @@ abstract class PlannerBase( objectIdentifier, tableToFind, Collections.emptyMap(), - getTableConfig.getConfiguration, + getTableConfig, getFlinkContext.getClassLoader, isTemporary) Option(resolvedTable, tableSink) @@ -434,39 +426,6 @@ abstract class PlannerBase( } } - /** - * Checks whether the [[CatalogTable]] uses legacy connector sink options. - */ - private def isLegacyConnectorOptions( - objectIdentifier: ObjectIdentifier, - catalogTable: CatalogTable, - isTemporary: Boolean) = { - // normalize option keys - val properties = new DescriptorProperties(true) - properties.putProperties(catalogTable.getOptions) - if (properties.containsKey(ConnectorDescriptorValidator.CONNECTOR_TYPE)) { - true - } else { - val catalog = catalogManager.getCatalog(objectIdentifier.getCatalogName) - try { - // try to create legacy table source using the options, - // some legacy factories uses the new 'connector' key - TableFactoryUtil.findAndCreateTableSink( - catalog.orElse(null), - objectIdentifier, - catalogTable, - getTableConfig.getConfiguration, - isStreamingMode, - isTemporary) - // success, then we will use the legacy factories - true - } catch { - // fail, then we will use new factories - case _: Throwable => false - } - } - } - protected def createSerdeContext: SerdeContext = { val planner = createFlinkPlanner new SerdeContext( @@ -479,22 +438,19 @@ abstract class PlannerBase( } protected def beforeTranslation(): Unit = { - val configuration = tableConfig.getConfiguration - // Add query start time to TableConfig, these config are used internally, // these configs will be used by temporal functions like CURRENT_TIMESTAMP,LOCALTIMESTAMP. val epochTime :JLong = System.currentTimeMillis() - configuration.set(TABLE_QUERY_START_EPOCH_TIME, epochTime) + tableConfig.set(TABLE_QUERY_START_EPOCH_TIME, epochTime) val localTime :JLong = epochTime + TimeZone.getTimeZone(tableConfig.getLocalTimeZone).getOffset(epochTime) - configuration.set(TABLE_QUERY_START_LOCAL_TIME, localTime) + tableConfig.set(TABLE_QUERY_START_LOCAL_TIME, localTime) - getExecEnv.configure( - configuration, - Thread.currentThread().getContextClassLoader) + // We pass only the configuration to avoid reconfiguration with the rootConfiguration + getExecEnv.configure(tableConfig.getConfiguration, Thread.currentThread().getContextClassLoader) // Use config parallelism to override env parallelism. - val defaultParallelism = getTableConfig.getConfiguration.getInteger( + val defaultParallelism = getTableConfig.get( ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM) if (defaultParallelism > 0) { getExecEnv.getConfig.setParallelism(defaultParallelism) @@ -548,6 +504,7 @@ abstract class PlannerBase( val transformations = translateToPlan(execGraph) afterTranslation() + // We pass only the configuration to avoid reconfiguration with the rootConfiguration val streamGraph = executor.createPipeline(transformations, tableConfig.getConfiguration, null) .asInstanceOf[StreamGraph] diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/StreamPlanner.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/StreamPlanner.scala index 6e896d84cef16..1c4a5126325c6 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/StreamPlanner.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/StreamPlanner.scala @@ -190,6 +190,7 @@ class StreamPlanner( val transformations = translateToPlan(execGraph) afterTranslation() + // We pass only the configuration to avoid reconfiguration with the rootConfiguration val streamGraph = executor.createPipeline(transformations, tableConfig.getConfiguration, null) .asInstanceOf[StreamGraph] @@ -210,7 +211,7 @@ class StreamPlanner( override def beforeTranslation(): Unit = { super.beforeTranslation() - val runtimeMode = getConfiguration.get(ExecutionOptions.RUNTIME_MODE) + val runtimeMode = getTableConfig.get(ExecutionOptions.RUNTIME_MODE) if (runtimeMode != RuntimeExecutionMode.STREAMING) { throw new IllegalArgumentException( "Mismatch between configured runtime mode and actual runtime mode. " + diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/fieldExpression.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/fieldExpression.scala index fd86f6749124f..013e687734ce8 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/fieldExpression.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/fieldExpression.scala @@ -20,7 +20,6 @@ package org.apache.flink.table.planner.expressions import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.api._ import org.apache.flink.table.operations.QueryOperation -import org.apache.flink.table.planner.calcite.FlinkRelBuilder.NamedWindowProperty import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.calcite.FlinkTypeFactory._ import org.apache.flink.table.planner.validate.{ValidationFailure, ValidationResult, ValidationSuccess} @@ -150,9 +149,6 @@ case class RowtimeAttribute(expr: PlannerExpression) extends TimeAttribute(expr) } } - override def toNamedWindowProperty(name: String): NamedWindowProperty = - NamedWindowProperty(name, this) - override def toString: String = s"rowtime($child)" } @@ -174,9 +170,6 @@ case class ProctimeAttribute(expr: PlannerExpression) extends TimeAttribute(expr override def resultType: TypeInformation[_] = TimeIndicatorTypeInfo.PROCTIME_INDICATOR - override def toNamedWindowProperty(name: String): NamedWindowProperty = - NamedWindowProperty(name, this) - override def toString: String = s"proctime($child)" } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/windowProperties.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/windowProperties.scala index 0e68163edd3d5..ce6940a58171f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/windowProperties.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/windowProperties.scala @@ -19,13 +19,10 @@ package org.apache.flink.table.planner.expressions import org.apache.flink.api.common.typeinfo.{SqlTimeTypeInfo, TypeInformation} -import org.apache.flink.table.planner.calcite.FlinkRelBuilder.NamedWindowProperty import org.apache.flink.table.planner.validate.{ValidationFailure, ValidationSuccess} trait WindowProperty { - def toNamedWindowProperty(name: String): NamedWindowProperty - def resultType: TypeInformation[_] } @@ -42,8 +39,6 @@ abstract class AbstractWindowProperty(child: PlannerExpression) } else { ValidationFailure("Child must be a window reference.") } - - def toNamedWindowProperty(name: String): NamedWindowProperty = NamedWindowProperty(name, this) } case class WindowStart(child: PlannerExpression) extends AbstractWindowProperty(child) { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnNullCount.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnNullCount.scala index 175e033539a24..00fc1b90148d5 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnNullCount.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnNullCount.scala @@ -158,8 +158,7 @@ class FlinkRelMdColumnNullCount private extends MetadataHandler[ColumnNullCount] // If predicate has $index is not null, null count of index is must be 0 after predicate. val rexBuilder = rel.getCluster.getRexBuilder val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(rel) - val maxCnfNodeCount = tableConfig.getConfiguration.getInteger( - FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT) + val maxCnfNodeCount = tableConfig.get(FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT) val cnf = FlinkRexUtil.toCnf(rexBuilder, maxCnfNodeCount, predicate) val conjunctions = RelOptUtil.conjunctions(cnf) val notNullPredicatesAtIndexField = conjunctions.exists { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCount.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCount.scala index a7a6c287a4abc..67b8a8c44de56 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCount.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCount.scala @@ -204,8 +204,7 @@ class FlinkRelMdDistinctRowCount private extends MetadataHandler[BuiltInMetadata } else { val rexBuilder = rel.getCluster.getRexBuilder val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(rel) - val maxCnfNodeCount = tableConfig.getConfiguration.getInteger( - FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT) + val maxCnfNodeCount = tableConfig.get(FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT) val cnf = FlinkRexUtil.toCnf(rexBuilder, maxCnfNodeCount, predicate) val conjunctions = RelOptUtil.conjunctions(cnf) val conjunctionsWithoutExpandId = conjunctions.filterNot { c => diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistribution.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistribution.scala index 046e221a449ab..4e1dde7577405 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistribution.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistribution.scala @@ -72,8 +72,7 @@ class FlinkRelMdDistribution private extends MetadataHandler[FlinkDistribution] def flinkDistribution(sort: Sort, mq: RelMetadataQuery): FlinkRelDistribution = { val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(sort) - val enableRangeSort = tableConfig.getConfiguration.getBoolean( - BatchPhysicalSortRule.TABLE_EXEC_RANGE_SORT_ENABLED) + val enableRangeSort = tableConfig.get(BatchPhysicalSortRule.TABLE_EXEC_RANGE_SORT_ENABLED) if ((sort.getCollation.getFieldCollations.nonEmpty && sort.fetch == null && sort.offset == null) && enableRangeSort) { //If Sort is global sort, and the table config allows the range partition. diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCount.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCount.scala index 5b9ee747f31c7..5c39e662d6b7b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCount.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCount.scala @@ -176,8 +176,7 @@ class FlinkRelMdRowCount private extends MetadataHandler[BuiltInMetadata.RowCoun val inputRowCnt = mq.getRowCount(input) val config = rel.getCluster.getPlanner.getContext.unwrap(classOf[FlinkContext]).getTableConfig val parallelism = (inputRowCnt / - config.getConfiguration.getLong( - FlinkRelMdRowCount.TABLE_OPTIMIZER_ROWS_PER_LOCALAGG) + 1).toInt + config.get(FlinkRelMdRowCount.TABLE_OPTIMIZER_ROWS_PER_LOCALAGG) + 1).toInt if (parallelism == 1) { ndvOfGroupKeysOnGlobalAgg } else if (grouping.isEmpty) { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimator.scala index f01ab7e940989..34632e82b9cdb 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimator.scala @@ -57,8 +57,7 @@ class SelectivityEstimator(rel: RelNode, mq: FlinkRelMetadataQuery) private val rexBuilder = rel.getCluster.getRexBuilder private val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(rel) - private val maxCnfNodeCount = tableConfig.getConfiguration.getInteger( - FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT) + private val maxCnfNodeCount = tableConfig.get(FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT) // these default values is referred to RelMdUtil#guessSelectivity private[flink] val defaultComparisonSelectivity = Some(0.5d) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalWindowAggregate.scala index 0d70fd7403b35..f3e35e14a4e10 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalWindowAggregate.scala @@ -36,7 +36,7 @@ final class LogicalWindowAggregate( groupSet: ImmutableBitSet, aggCalls: util.List[AggregateCall], window: LogicalWindow, - namedProperties: Seq[NamedWindowProperty]) + namedProperties: util.List[NamedWindowProperty]) extends WindowAggregate(cluster, traitSet, child, groupSet, aggCalls, window, namedProperties) { override def copy( @@ -55,7 +55,7 @@ final class LogicalWindowAggregate( namedProperties) } - def copy(namedProperties: Seq[NamedWindowProperty]): LogicalWindowAggregate = { + def copy(namedProperties: util.List[NamedWindowProperty]): LogicalWindowAggregate = { new LogicalWindowAggregate( cluster, traitSet, @@ -71,7 +71,7 @@ object LogicalWindowAggregate { def create( window: LogicalWindow, - namedProperties: Seq[NamedWindowProperty], + namedProperties: util.List[NamedWindowProperty], agg: Aggregate): LogicalWindowAggregate = { require(agg.getGroupType == Group.SIMPLE) val cluster: RelOptCluster = agg.getCluster diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalWindowTableAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalWindowTableAggregate.scala index 6ae042f6185f8..f9a2ed286f406 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalWindowTableAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalWindowTableAggregate.scala @@ -41,7 +41,7 @@ class LogicalWindowTableAggregate( groupSets: util.List[ImmutableBitSet], aggCalls: util.List[AggregateCall], window: LogicalWindow, - namedProperties: Seq[NamedWindowProperty]) + namedProperties: util.List[NamedWindowProperty]) extends WindowTableAggregate( cluster, traitSet, @@ -69,7 +69,7 @@ object LogicalWindowTableAggregate { def create( window: LogicalWindow, - namedProperties: Seq[NamedWindowProperty], + namedProperties: util.List[NamedWindowProperty], aggregate: Aggregate): LogicalWindowTableAggregate = { val cluster: RelOptCluster = aggregate.getCluster diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/WindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/WindowAggregate.scala index 884be0ab90c5c..c28dd1a732305 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/WindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/WindowAggregate.scala @@ -31,6 +31,9 @@ import org.apache.calcite.util.ImmutableBitSet import java.util +import scala.collection.JavaConverters._ + + /** * Relational operator that eliminates duplicates and computes totals with time window group. * @@ -43,7 +46,7 @@ abstract class WindowAggregate( groupSet: ImmutableBitSet, aggCalls: util.List[AggregateCall], window: LogicalWindow, - namedProperties: Seq[NamedWindowProperty]) + namedProperties: util.List[NamedWindowProperty]) extends Aggregate( cluster, traitSet, @@ -54,7 +57,7 @@ abstract class WindowAggregate( def getWindow: LogicalWindow = window - def getNamedProperties: Seq[NamedWindowProperty] = namedProperties + def getNamedProperties: util.List[NamedWindowProperty] = namedProperties override def accept(shuttle: RelShuttle): RelNode = shuttle.visit(this) @@ -63,7 +66,7 @@ abstract class WindowAggregate( val typeFactory = getCluster.getTypeFactory.asInstanceOf[FlinkTypeFactory] val builder = typeFactory.builder builder.addAll(aggregateRowType.getFieldList) - namedProperties.foreach { namedProp => + namedProperties.asScala.foreach { namedProp => builder.add( namedProp.getName, typeFactory.createFieldTypeFromLogicalType(namedProp.getProperty.getResultType) @@ -82,6 +85,6 @@ abstract class WindowAggregate( override def explainTerms(pw: RelWriter): RelWriter = { super.explainTerms(pw) .item("window", window) - .item("properties", namedProperties.map(_.getName).mkString(", ")) + .item("properties", namedProperties.asScala.map(_.getName).mkString(", ")) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/WindowTableAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/WindowTableAggregate.scala index 98bcaea381f04..d388b5b313d3e 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/WindowTableAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/WindowTableAggregate.scala @@ -30,6 +30,8 @@ import org.apache.calcite.util.ImmutableBitSet import java.util +import scala.collection.JavaConverters._ + /** * Relational operator that represents a window table aggregate. A TableAggregate is similar to the * [[org.apache.calcite.rel.core.Aggregate]] but may output 0 or more records for a group. @@ -42,19 +44,19 @@ abstract class WindowTableAggregate( groupSets: util.List[ImmutableBitSet], aggCalls: util.List[AggregateCall], window: LogicalWindow, - namedProperties: Seq[NamedWindowProperty]) + namedProperties: util.List[NamedWindowProperty]) extends TableAggregate(cluster, traitSet, input, groupSet, groupSets, aggCalls) { def getWindow: LogicalWindow = window - def getNamedProperties: Seq[NamedWindowProperty] = namedProperties + def getNamedProperties: util.List[NamedWindowProperty] = namedProperties override def deriveRowType(): RelDataType = { val aggregateRowType = super.deriveRowType() val typeFactory = getCluster.getTypeFactory.asInstanceOf[FlinkTypeFactory] val builder = typeFactory.builder builder.addAll(aggregateRowType.getFieldList) - namedProperties.foreach { namedProp => + namedProperties.asScala.foreach { namedProp => builder.add( namedProp.getName, typeFactory.createFieldTypeFromLogicalType(namedProp.getProperty.getResultType) @@ -66,6 +68,6 @@ abstract class WindowTableAggregate( override def explainTerms(pw: RelWriter): RelWriter = { super.explainTerms(pw) .item("window", window) - .item("properties", namedProperties.map(_.getName).mkString(", ")) + .item("properties", namedProperties.asScala.map(_.getName).mkString(", ")) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalSort.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalSort.scala index b0db94d242ded..4b4f5b3a270cc 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalSort.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalSort.scala @@ -105,8 +105,8 @@ class FlinkLogicalSortBatchConverter extends ConverterRule( val sort = rel.asInstanceOf[LogicalSort] val newInput = RelOptRule.convert(sort.getInput, FlinkConventions.LOGICAL) val config = sort.getCluster.getPlanner.getContext.unwrap(classOf[FlinkContext]).getTableConfig - val enableRangeSort = config.getConfiguration.getBoolean(TABLE_EXEC_RANGE_SORT_ENABLED) - val limitValue = config.getConfiguration.getInteger(TABLE_EXEC_SORT_DEFAULT_LIMIT) + val enableRangeSort = config.get(TABLE_EXEC_RANGE_SORT_ENABLED) + val limitValue = config.get(TABLE_EXEC_SORT_DEFAULT_LIMIT) val (offset, fetch) = if (sort.fetch == null && sort.offset == null && !enableRangeSort && limitValue > 0) { //force the sort add limit diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalWindowAggregate.scala index 8cef1170aee3c..7884f59d7fd51 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalWindowAggregate.scala @@ -43,7 +43,7 @@ class FlinkLogicalWindowAggregate( groupSet: ImmutableBitSet, aggCalls: util.List[AggregateCall], window: LogicalWindow, - namedProperties: Seq[NamedWindowProperty]) + namedProperties: util.List[NamedWindowProperty]) extends WindowAggregate(cluster, traitSet, child, groupSet, aggCalls, window, namedProperties) with FlinkLogicalRel { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalWindowTableAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalWindowTableAggregate.scala index add5d4149e04d..7dcc81bc12a9e 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalWindowTableAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalWindowTableAggregate.scala @@ -44,7 +44,7 @@ class FlinkLogicalWindowTableAggregate( groupSets: util.List[ImmutableBitSet], aggCalls: util.List[AggregateCall], window: LogicalWindow, - namedProperties: Seq[NamedWindowProperty]) + namedProperties: util.List[NamedWindowProperty]) extends WindowTableAggregate( cluster, traitSet, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalBoundedStreamScan.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalBoundedStreamScan.scala index 12282936c2f0b..8122033837cac 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalBoundedStreamScan.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalBoundedStreamScan.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNode import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecBoundedStreamScan import org.apache.flink.table.planner.plan.schema.DataStreamTable import org.apache.flink.table.planner.plan.utils.RelExplainUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan._ import org.apache.calcite.rel.`type`.RelDataType @@ -71,6 +72,7 @@ class BatchPhysicalBoundedStreamScan( override def translateToExecNode(): ExecNode[_] = { new BatchExecBoundedStreamScan( + unwrapTableConfig(this), boundedStreamTable.dataStream, boundedStreamTable.dataType, boundedStreamTable.fieldIndexes, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalCalc.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalCalc.scala index 6fefa39f436e8..2c62493ce4dd7 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalCalc.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalCalc.scala @@ -20,7 +20,8 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecCalc -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan._ import org.apache.calcite.rel._ @@ -54,6 +55,7 @@ class BatchPhysicalCalc( } new BatchExecCalc( + unwrapTableConfig(this), projection, condition, InputProperty.DEFAULT, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalCorrelate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalCorrelate.scala index f6dc6105e16f6..6ba5e703e1b61 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalCorrelate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalCorrelate.scala @@ -19,9 +19,10 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecCorrelate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableFunctionScan import org.apache.flink.table.planner.plan.utils.JoinTypeUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -65,6 +66,7 @@ class BatchPhysicalCorrelate( override def translateToExecNode(): ExecNode[_] = { new BatchExecCorrelate( + unwrapTableConfig(this), JoinTypeUtil.getFlinkJoinType(joinType), scan.getCall.asInstanceOf[RexCall], condition.orNull, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalExchange.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalExchange.scala index e58063f96127c..cd28172d21621 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalExchange.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalExchange.scala @@ -49,6 +49,7 @@ class BatchPhysicalExchange( override def translateToExecNode(): ExecNode[_] = { new BatchExecExchange( + unwrapTableConfig(this), getInputProperty, FlinkTypeFactory.toLogicalRowType(getRowType), getRelDetailedDescription) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalExpand.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalExpand.scala index 237929433b7cb..99905c0b8f99f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalExpand.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalExpand.scala @@ -21,7 +21,8 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.calcite.Expand import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecExpand -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -53,10 +54,10 @@ class BatchPhysicalExpand( override def translateToExecNode(): ExecNode[_] = { new BatchExecExpand( + unwrapTableConfig(this), projects, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashAggregate.scala index d5ebb609cf36f..d0cc5bfacd49a 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashAggregate.scala @@ -22,9 +22,10 @@ import org.apache.flink.table.functions.UserDefinedFunction import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecHashAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.rules.physical.batch.BatchPhysicalJoinRuleBase import org.apache.flink.table.planner.plan.utils.{FlinkRelOptUtil, RelExplainUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptRule, RelTraitSet} import org.apache.calcite.rel.RelDistribution.Type.{HASH_DISTRIBUTED, SINGLETON} @@ -110,7 +111,7 @@ class BatchPhysicalHashAggregate( } else { // If partialKey is enabled, try to use partial key to satisfy the required distribution val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(this) - val partialKeyEnabled = tableConfig.getConfiguration.getBoolean( + val partialKeyEnabled = tableConfig.get( BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED) partialKeyEnabled && groupKeysList.containsAll(shuffleKeys) } @@ -148,6 +149,7 @@ class BatchPhysicalHashAggregate( InputProperty.hashDistribution(grouping) } new BatchExecHashAggregate( + unwrapTableConfig(this), grouping, auxGrouping, getAggCallList.toArray, @@ -159,7 +161,6 @@ class BatchPhysicalHashAggregate( .damBehavior(InputProperty.DamBehavior.END_INPUT) .build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashJoin.scala index 1606874fe7dde..708208128448c 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashJoin.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.planner.plan.nodes.FlinkConventions import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecHashJoin import org.apache.flink.table.planner.plan.utils.{FlinkRelMdUtil, JoinUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.operators.join.HashJoinType import org.apache.flink.table.runtime.typeutils.BinaryRowDataSerializer @@ -166,18 +167,18 @@ class BatchPhysicalHashJoin( val rightRowCount = Util.first(mq.getRowCount(right), 200000).toLong val (leftEdge, rightEdge) = getInputProperties new BatchExecHashJoin( - joinSpec, - leftRowSize, - rightRowSize, - leftRowCount, - rightRowCount, - leftIsBuild, - tryDistinctBuildRow, - leftEdge, - rightEdge, - FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + unwrapTableConfig(this), + joinSpec, + leftRowSize, + rightRowSize, + leftRowCount, + rightRowCount, + leftIsBuild, + tryDistinctBuildRow, + leftEdge, + rightEdge, + FlinkTypeFactory.toLogicalRowType(getRowType), + getRelDetailedDescription) } private def getInputProperties: (InputProperty, InputProperty) = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashWindowAggregate.scala index 5838941603f35..b47f36b2d2216 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashWindowAggregate.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.logical.LogicalWindow import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecHashWindowAggregate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -89,6 +90,7 @@ class BatchPhysicalHashWindowAggregate( InputProperty.hashDistribution(grouping) } new BatchExecHashWindowAggregate( + unwrapTableConfig(this), grouping, auxGrouping, getAggCallList.toArray, @@ -105,7 +107,6 @@ class BatchPhysicalHashWindowAggregate( .damBehavior(InputProperty.DamBehavior.END_INPUT) .build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacySink.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacySink.scala index fcdb53a9591d3..28d1445493e5b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacySink.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacySink.scala @@ -22,6 +22,7 @@ import org.apache.flink.table.planner.plan.nodes.calcite.LegacySink import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecLegacySink import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.UpdatingPlanChecker +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType import org.apache.flink.table.sinks.{TableSink, UpsertStreamTableSink} @@ -57,13 +58,13 @@ class BatchPhysicalLegacySink[T]( case _ => Option.empty[Array[String]] } new BatchExecLegacySink[T]( + unwrapTableConfig(this), sink, upsertKeys.orNull, // the input records will not trigger any output of a sink because it has no output, // so it's dam behavior is BLOCKING InputProperty.builder().damBehavior(InputProperty.DamBehavior.BLOCKING).build(), fromDataTypeToLogicalType(sink.getConsumedDataType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacyTableSourceScan.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacyTableSourceScan.scala index 8778505f8a1ab..be480d501148d 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacyTableSourceScan.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacyTableSourceScan.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNode import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecLegacyTableSourceScan import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalLegacyTableSourceScan import org.apache.flink.table.planner.plan.schema.LegacyTableSourceTable +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.sources.StreamTableSource import org.apache.calcite.plan._ @@ -61,6 +62,7 @@ class BatchPhysicalLegacyTableSourceScan( override def translateToExecNode(): ExecNode[_] = { new BatchExecLegacyTableSourceScan( + unwrapTableConfig(this), tableSource, getTable.getQualifiedName, FlinkTypeFactory.toLogicalRowType(getRowType), diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLimit.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLimit.scala index d3ee2fa4026d3..eee744c6d5ad1 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLimit.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLimit.scala @@ -22,9 +22,10 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.cost.FlinkCost._ import org.apache.flink.table.planner.plan.cost.FlinkCostFactory import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecLimit -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.RelExplainUtil.fetchToString import org.apache.flink.table.planner.plan.utils.SortUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet} import org.apache.calcite.rel._ @@ -81,11 +82,12 @@ class BatchPhysicalLimit( override def translateToExecNode(): ExecNode[_] = { new BatchExecLimit( - limitStart, - limitEnd, - isGlobal, - InputProperty.DEFAULT, - FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription) + unwrapTableConfig(this), + limitStart, + limitEnd, + isGlobal, + InputProperty.DEFAULT, + FlinkTypeFactory.toLogicalRowType(getRowType), + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalHashAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalHashAggregate.scala index 98e3da6fed37f..0359472912356 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalHashAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalHashAggregate.scala @@ -22,8 +22,9 @@ import org.apache.flink.table.functions.UserDefinedFunction import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecHashAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.RelExplainUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptRule, RelTraitSet} import org.apache.calcite.rel.RelDistribution.Type @@ -121,6 +122,7 @@ class BatchPhysicalLocalHashAggregate( override def translateToExecNode(): ExecNode[_] = { new BatchExecHashAggregate( + unwrapTableConfig(this), grouping, auxGrouping, getAggCallList.toArray, @@ -129,8 +131,7 @@ class BatchPhysicalLocalHashAggregate( false, // isFinal is always false getInputProperty, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } private def getInputProperty: InputProperty = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalHashWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalHashWindowAggregate.scala index 7d81fe048f2a9..cf3a735ac17fd 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalHashWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalHashWindowAggregate.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.logical.LogicalWindow import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecHashWindowAggregate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -82,6 +83,7 @@ class BatchPhysicalLocalHashWindowAggregate( override def translateToExecNode(): ExecNode[_] = { new BatchExecHashWindowAggregate( + unwrapTableConfig(this), grouping, auxGrouping, getAggCallList.toArray, @@ -95,7 +97,6 @@ class BatchPhysicalLocalHashWindowAggregate( false, // isFinal is always false InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalSortAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalSortAggregate.scala index 59faaf0483ca7..93cf6b8ada28f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalSortAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalSortAggregate.scala @@ -22,8 +22,9 @@ import org.apache.flink.table.functions.UserDefinedFunction import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecSortAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{FlinkRelOptUtil, RelExplainUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptRule, RelTraitSet} import org.apache.calcite.rel.RelDistribution.Type @@ -129,6 +130,7 @@ class BatchPhysicalLocalSortAggregate( override def translateToExecNode(): ExecNode[_] = { new BatchExecSortAggregate( + unwrapTableConfig(this), grouping, auxGrouping, getAggCallList.toArray, @@ -137,8 +139,7 @@ class BatchPhysicalLocalSortAggregate( false, // isFinal is always false getInputProperty, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } private def getInputProperty: InputProperty = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalSortWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalSortWindowAggregate.scala index 9a5fdca397f09..05a1902a1ccd7 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalSortWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalSortWindowAggregate.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.logical.LogicalWindow import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecSortWindowAggregate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -80,6 +81,7 @@ class BatchPhysicalLocalSortWindowAggregate( override def translateToExecNode(): ExecNode[_] = { new BatchExecSortWindowAggregate( + unwrapTableConfig(this), grouping, auxGrouping, getAggCallList.toArray, @@ -93,7 +95,6 @@ class BatchPhysicalLocalSortWindowAggregate( false, // isFinal is always false InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLookupJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLookupJoin.scala index e66a32c45368d..900e19fd0e35a 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLookupJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLookupJoin.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalLookupJoin import org.apache.flink.table.planner.plan.utils.{FlinkRexUtil, JoinTypeUtil} import org.apache.flink.table.planner.utils.JavaScalaConversionUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptTable, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -77,6 +78,7 @@ class BatchPhysicalLookupJoin( } new BatchExecLookupJoin( + unwrapTableConfig(this), JoinTypeUtil.getFlinkJoinType(joinType), remainingCondition.orNull, new TemporalTableSourceSpec(temporalTable), diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalNestedLoopJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalNestedLoopJoin.scala index 6faa2b92434a3..93119021dc464 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalNestedLoopJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalNestedLoopJoin.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.cost.{FlinkCost, FlinkCostFactory} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecNestedLoopJoin import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.JoinTypeUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.typeutils.BinaryRowDataSerializer import org.apache.calcite.plan._ @@ -114,6 +115,7 @@ class BatchPhysicalNestedLoopJoin( override def translateToExecNode(): ExecNode[_] = { val (leftInputProperty, rightInputProperty) = getInputProperties new BatchExecNestedLoopJoin( + unwrapTableConfig(this), JoinTypeUtil.getFlinkJoinType(joinType), condition, leftIsBuild, @@ -121,8 +123,7 @@ class BatchPhysicalNestedLoopJoin( leftInputProperty, rightInputProperty, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } def getInputProperties: (InputProperty, InputProperty) = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalOverAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalOverAggregate.scala index 5260dfe6e0d93..a45481462627f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalOverAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalOverAggregate.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecOverAggrega import org.apache.flink.table.planner.plan.nodes.exec.spec.{OverSpec, PartitionSpec} import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.OverAggregateUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan._ import org.apache.calcite.rel._ @@ -71,6 +72,7 @@ class BatchPhysicalOverAggregate( InputProperty.hashDistribution(partitionKeyIndices) } new BatchExecOverAggregate( + unwrapTableConfig(this), new OverSpec( new PartitionSpec(partitionKeyIndices), offsetAndInsensitiveSensitiveGroups.map(OverAggregateUtil.createGroupSpec(_, logicWindow)), @@ -78,7 +80,6 @@ class BatchPhysicalOverAggregate( OverAggregateUtil.calcOriginalInputFields(logicWindow)), InputProperty.builder().requiredDistribution(requiredDistribution).build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalOverAggregateBase.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalOverAggregateBase.scala index 6c62e0848337d..675e15eebf9a5 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalOverAggregateBase.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalOverAggregateBase.scala @@ -137,7 +137,7 @@ abstract class BatchPhysicalOverAggregateBase( val isAllFieldsFromInput = requiredDistribution.getKeys.forall(_ < inputFieldCnt) if (isAllFieldsFromInput) { val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(this) - if (tableConfig.getConfiguration.getBoolean( + if (tableConfig.get( BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED)) { ImmutableIntList.of(partitionKeyIndices: _*).containsAll(requiredDistribution.getKeys) } else { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonCalc.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonCalc.scala index a32457ba7da71..0fa260bf346ac 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonCalc.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonCalc.scala @@ -22,6 +22,7 @@ import org.apache.flink.table.api.TableException import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecPythonCalc import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -58,6 +59,7 @@ class BatchPhysicalPythonCalc( } new BatchExecPythonCalc( + unwrapTableConfig(this), projection, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonCorrelate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonCorrelate.scala index 298476c0ce1df..3d59cea816f5d 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonCorrelate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonCorrelate.scala @@ -20,9 +20,10 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch import org.apache.flink.table.api.TableException import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecPythonCorrelate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableFunctionScan import org.apache.flink.table.planner.plan.utils.JoinTypeUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -74,11 +75,11 @@ class BatchPhysicalPythonCorrelate( } new BatchExecPythonCorrelate( + unwrapTableConfig(this), JoinTypeUtil.getFlinkJoinType(joinType), scan.getCall.asInstanceOf[RexCall], InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupAggregate.scala index 9d427c0f8d2b0..136c7260cb2ea 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupAggregate.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecPythonGroupAggregate import org.apache.flink.table.planner.plan.rules.physical.batch.BatchPhysicalJoinRuleBase import org.apache.flink.table.planner.plan.utils.{FlinkRelOptUtil, RelExplainUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptRule, RelTraitSet} import org.apache.calcite.rel.RelDistribution.Type.{HASH_DISTRIBUTED, SINGLETON} @@ -92,8 +93,8 @@ class BatchPhysicalPythonGroupAggregate( true } else { // If partialKey is enabled, try to use partial key to satisfy the required distribution - val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(this) - val partialKeyEnabled = tableConfig.getConfiguration.getBoolean( + val tableConfig = unwrapTableConfig(this) + val partialKeyEnabled = tableConfig.get( BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED) partialKeyEnabled && groupKeysList.containsAll(shuffleKeys) } @@ -156,6 +157,7 @@ class BatchPhysicalPythonGroupAggregate( InputProperty.hashDistribution(grouping) } new BatchExecPythonGroupAggregate( + unwrapTableConfig(this), grouping, grouping ++ auxGrouping, aggCalls.toArray, @@ -164,8 +166,7 @@ class BatchPhysicalPythonGroupAggregate( .damBehavior(InputProperty.DamBehavior.END_INPUT) .build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupWindowAggregate.scala index ae2c2f57b9373..09e9fcdf9b746 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupWindowAggregate.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.planner.plan.cost.{FlinkCost, FlinkCostFactory} import org.apache.flink.table.planner.plan.logical.LogicalWindow import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecPythonGroupWindowAggregate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet} @@ -104,6 +105,7 @@ class BatchPhysicalPythonGroupWindowAggregate( InputProperty.hashDistribution(grouping) } new BatchExecPythonGroupWindowAggregate( + unwrapTableConfig(this), grouping, grouping ++ auxGrouping, aggCalls.toArray, @@ -112,7 +114,6 @@ class BatchPhysicalPythonGroupWindowAggregate( namedWindowProperties.toArray, InputProperty.builder().requiredDistribution(requiredDistribution).build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonOverAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonOverAggregate.scala index 9f42ab41ee2ed..63535c045f0a9 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonOverAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonOverAggregate.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecPythonOverA import org.apache.flink.table.planner.plan.nodes.exec.spec.{OverSpec, PartitionSpec} import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.OverAggregateUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan._ import org.apache.calcite.rel._ @@ -70,6 +71,7 @@ class BatchPhysicalPythonOverAggregate( InputProperty.hashDistribution(partitionKeyIndices) } new BatchExecPythonOverAggregate( + unwrapTableConfig(this), new OverSpec( new PartitionSpec(partitionKeyIndices), offsetAndInsensitiveSensitiveGroups.map(OverAggregateUtil.createGroupSpec(_, logicWindow)), @@ -77,7 +79,6 @@ class BatchPhysicalPythonOverAggregate( OverAggregateUtil.calcOriginalInputFields(logicWindow)), InputProperty.builder().requiredDistribution(requiredDistribution).build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalRank.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalRank.scala index 4e8649f5ad31f..18626e783dcac 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalRank.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalRank.scala @@ -27,6 +27,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecRank import org.apache.flink.table.planner.plan.rules.physical.batch.BatchPhysicalJoinRuleBase import org.apache.flink.table.planner.plan.utils.{FlinkRelOptUtil, RelExplainUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.operators.rank.{ConstantRankRange, RankRange, RankType} import org.apache.calcite.plan._ @@ -135,7 +136,7 @@ class BatchPhysicalRank( } else { // If partialKey is enabled, try to use partial key to satisfy the required distribution val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(this) - val partialKeyEnabled = tableConfig.getConfiguration.getBoolean( + val partialKeyEnabled = tableConfig.get( BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED) partialKeyEnabled && partitionKeyList.containsAll(shuffleKeys) } @@ -228,12 +229,17 @@ class BatchPhysicalRank( } override def translateToExecNode(): ExecNode[_] = { - val requiredDistribution = if (partitionKey.length() == 0) { - InputProperty.SINGLETON_DISTRIBUTION + val requiredDistribution = if (isGlobal) { + if (partitionKey.length() == 0) { + InputProperty.SINGLETON_DISTRIBUTION + } else { + InputProperty.hashDistribution(partitionKey.toArray) + } } else { - InputProperty.hashDistribution(partitionKey.toArray) + InputProperty.UNKNOWN_DISTRIBUTION } new BatchExecRank( + unwrapTableConfig(this), partitionKey.toArray, orderKey.getFieldCollations.map(_.getFieldIndex).toArray, rankStart, @@ -241,7 +247,6 @@ class BatchPhysicalRank( outputRankNumber, InputProperty.builder().requiredDistribution(requiredDistribution).build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSink.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSink.scala index 1400b0990aad1..566fae152bc0f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSink.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSink.scala @@ -18,7 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch -import org.apache.flink.table.catalog.{CatalogTable, ContextResolvedTable, ObjectIdentifier, ResolvedCatalogTable} +import org.apache.flink.table.catalog.ContextResolvedTable import org.apache.flink.table.connector.sink.DynamicTableSink import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.abilities.sink.SinkAbilitySpec @@ -26,7 +26,7 @@ import org.apache.flink.table.planner.plan.nodes.calcite.Sink import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecSink import org.apache.flink.table.planner.plan.nodes.exec.spec.DynamicTableSinkSpec import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} -import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -67,12 +67,12 @@ class BatchPhysicalSink( tableSinkSpec.setTableSink(tableSink) new BatchExecSink( + unwrapTableConfig(this), tableSinkSpec, // the input records will not trigger any output of a sink because it has no output, // so it's dam behavior is BLOCKING InputProperty.builder().damBehavior(InputProperty.DamBehavior.BLOCKING).build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSort.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSort.scala index c1ac0b732eaa2..73236ee2d551d 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSort.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSort.scala @@ -21,8 +21,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.cost.{FlinkCost, FlinkCostFactory} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecSort -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{FlinkRelMdUtil, RelExplainUtil, SortUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet} import org.apache.calcite.rel.core.Sort @@ -79,10 +80,10 @@ class BatchPhysicalSort( @Override override def translateToExecNode(): ExecNode[_] = { new BatchExecSort( + unwrapTableConfig(this), SortUtil.getSortSpec(sortCollation.getFieldCollations), InputProperty.builder().damBehavior(InputProperty.DamBehavior.END_INPUT).build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortAggregate.scala index 71b0fcf02a0b0..cd9ebd078775d 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortAggregate.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecSortAggrega import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} import org.apache.flink.table.planner.plan.rules.physical.batch.BatchPhysicalJoinRuleBase import org.apache.flink.table.planner.plan.utils.{FlinkRelOptUtil, RelExplainUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptRule, RelTraitSet} import org.apache.calcite.rel.RelDistribution.Type.{HASH_DISTRIBUTED, SINGLETON} @@ -111,7 +112,7 @@ class BatchPhysicalSortAggregate( } else { // If partialKey is enabled, try to use partial key to satisfy the required distribution val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(this) - val partialKeyEnabled = tableConfig.getConfiguration.getBoolean( + val partialKeyEnabled = tableConfig.get( BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED) partialKeyEnabled && groupKeysList.containsAll(shuffleKeys) } @@ -155,6 +156,7 @@ class BatchPhysicalSortAggregate( override def translateToExecNode(): ExecNode[_] = { new BatchExecSortAggregate( + unwrapTableConfig(this), grouping, auxGrouping, getAggCallList.toArray, @@ -163,8 +165,7 @@ class BatchPhysicalSortAggregate( true, // isFinal is always true getInputProperty, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } private def getInputProperty: InputProperty = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortLimit.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortLimit.scala index 16228f1a72987..861d7d9811bf0 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortLimit.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortLimit.scala @@ -21,8 +21,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.cost.{FlinkCost, FlinkCostFactory} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecSortLimit -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{RelExplainUtil, SortUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet} import org.apache.calcite.rel.core.Sort @@ -100,13 +101,13 @@ class BatchPhysicalSortLimit( override def translateToExecNode(): ExecNode[_] = { new BatchExecSortLimit( + unwrapTableConfig(this), SortUtil.getSortSpec(sortCollation.getFieldCollations), limitStart, limitEnd, isGlobal, InputProperty.builder().damBehavior(InputProperty.DamBehavior.END_INPUT).build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortMergeJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortMergeJoin.scala index c2bfa50550f47..909489de3459b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortMergeJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortMergeJoin.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.planner.plan.cost.{FlinkCost, FlinkCostFactory} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecSortMergeJoin import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{FlinkRelMdUtil, FlinkRelOptUtil, JoinTypeUtil, JoinUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.operators.join.FlinkJoinType import org.apache.calcite.plan._ @@ -173,6 +174,7 @@ class BatchPhysicalSortMergeJoin( FlinkTypeFactory.toLogicalRowType(right.getRowType)) new BatchExecSortMergeJoin( + unwrapTableConfig(this), JoinTypeUtil.getFlinkJoinType(joinType), joinSpec.getLeftKeys, joinSpec.getRightKeys, @@ -188,8 +190,7 @@ class BatchPhysicalSortMergeJoin( .damBehavior(InputProperty.DamBehavior.END_INPUT) .build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } private def estimateOutputSize(relNode: RelNode): Double = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortWindowAggregate.scala index a013e283e6d4d..f48e51e20f354 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortWindowAggregate.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.logical.LogicalWindow import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecSortWindowAggregate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -85,6 +86,7 @@ class BatchPhysicalSortWindowAggregate( InputProperty.hashDistribution(grouping) } new BatchExecSortWindowAggregate( + unwrapTableConfig(this), grouping, auxGrouping, getAggCallList.toArray, @@ -98,7 +100,6 @@ class BatchPhysicalSortWindowAggregate( true, // isFinal is always true InputProperty.builder().requiredDistribution(requiredDistribution).build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalTableSourceScan.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalTableSourceScan.scala index e5f0ad2c7f602..7ab2bf7f7c945 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalTableSourceScan.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalTableSourceScan.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecTableSource import org.apache.flink.table.planner.plan.nodes.exec.spec.DynamicTableSourceSpec import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalTableSourceScan import org.apache.flink.table.planner.plan.schema.TableSourceTable +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan._ import org.apache.calcite.rel.RelNode @@ -72,6 +73,7 @@ class BatchPhysicalTableSourceScan( tableSourceSpec.setTableSource(tableSourceTable.tableSource) new BatchExecTableSourceScan( + unwrapTableConfig(this), tableSourceSpec, FlinkTypeFactory.toLogicalRowType(getRowType), getRelDetailedDescription) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalUnion.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalUnion.scala index 9d1d9986f0524..a440b01c71c66 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalUnion.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalUnion.scala @@ -21,7 +21,8 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecUnion -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptRule, RelTraitSet} import org.apache.calcite.rel.RelDistribution.Type._ @@ -95,9 +96,9 @@ class BatchPhysicalUnion( override def translateToExecNode(): ExecNode[_] = { new BatchExecUnion( + unwrapTableConfig(this), getInputs.map(_ => InputProperty.DEFAULT), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalValues.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalValues.scala index 23ff9dc89e12a..fbbcef7edc041 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalValues.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalValues.scala @@ -21,6 +21,7 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.ExecNode import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecValues +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import com.google.common.collect.ImmutableList import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -57,9 +58,9 @@ class BatchPhysicalValues( override def translateToExecNode(): ExecNode[_] = { new BatchExecValues( + unwrapTableConfig(this), tuples.asList().map(_.asList()), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalWindowTableFunction.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalWindowTableFunction.scala index 0c888332bd951..1b75542827333 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalWindowTableFunction.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalWindowTableFunction.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.logical.TimeAttributeWindowingStrateg import org.apache.flink.table.planner.plan.nodes.common.CommonPhysicalWindowTableFunction import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecWindowTableFunction import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -58,11 +59,11 @@ class BatchPhysicalWindowTableFunction( override def translateToExecNode(): ExecNode[_] = { new BatchExecWindowTableFunction( + unwrapTableConfig(this), windowing, // TODO set semantic window (such as session window) require other Dam Behavior InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalCalc.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalCalc.scala index 6603e1b515b3b..396765d8adc50 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalCalc.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalCalc.scala @@ -21,6 +21,7 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -54,6 +55,7 @@ class StreamPhysicalCalc( } new StreamExecCalc( + unwrapTableConfig(this), projection, condition, InputProperty.DEFAULT, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalChangelogNormalize.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalChangelogNormalize.scala index ce06e55379f65..7b5ff0e45981f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalChangelogNormalize.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalChangelogNormalize.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecChangelogNormalize import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -67,6 +68,7 @@ class StreamPhysicalChangelogNormalize( override def translateToExecNode(): ExecNode[_] = { val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) new StreamExecChangelogNormalize( + unwrapTableConfig(this), uniqueKeys, generateUpdateBefore, InputProperty.DEFAULT, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalCorrelate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalCorrelate.scala index 710c4df7959fa..5fbdfc33b0307 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalCorrelate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalCorrelate.scala @@ -19,9 +19,10 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCorrelate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableFunctionScan import org.apache.flink.table.planner.plan.utils.JoinTypeUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -65,6 +66,7 @@ class StreamPhysicalCorrelate( override def translateToExecNode(): ExecNode[_] = { new StreamExecCorrelate( + unwrapTableConfig(this), JoinTypeUtil.getFlinkJoinType(joinType), scan.getCall.asInstanceOf[RexCall], condition.orNull, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDataStreamScan.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDataStreamScan.scala index c80cc0d40fbbb..d3b95bf5575b9 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDataStreamScan.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDataStreamScan.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNode import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecDataStreamScan import org.apache.flink.table.planner.plan.schema.DataStreamTable import org.apache.flink.table.planner.plan.utils.RelExplainUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan._ import org.apache.calcite.rel.`type`.RelDataType @@ -73,6 +74,7 @@ class StreamPhysicalDataStreamScan( override def translateToExecNode(): ExecNode[_] = { new StreamExecDataStreamScan( + unwrapTableConfig(this), dataStreamTable.dataStream, dataStreamTable.dataType, dataStreamTable.fieldIndexes, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDeduplicate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDeduplicate.scala index ddc772ca68848..ae1cabcfc97a0 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDeduplicate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDeduplicate.scala @@ -20,8 +20,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecDeduplicate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -73,13 +74,13 @@ class StreamPhysicalDeduplicate( override def translateToExecNode(): ExecNode[_] = { val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) new StreamExecDeduplicate( + unwrapTableConfig(this), uniqueKeys, isRowtime, keepLastRow, generateUpdateBefore, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDropUpdateBefore.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDropUpdateBefore.scala index ba24f6e64cf83..3a9e75209e37e 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDropUpdateBefore.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDropUpdateBefore.scala @@ -20,8 +20,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecDropUpdateBefore -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -60,9 +61,9 @@ class StreamPhysicalDropUpdateBefore( } new StreamExecDropUpdateBefore( + unwrapTableConfig(this), InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalExchange.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalExchange.scala index e07caee902c9e..91c272cc8753b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalExchange.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalExchange.scala @@ -20,8 +20,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalExchange +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.{RelDistribution, RelNode} @@ -48,6 +49,7 @@ class StreamPhysicalExchange( override def translateToExecNode(): ExecNode[_] = { new StreamExecExchange( + unwrapTableConfig(this), InputProperty.builder.requiredDistribution(getRequiredDistribution).build, FlinkTypeFactory.toLogicalRowType(getRowType), getRelDetailedDescription) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalExpand.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalExpand.scala index 9e0c414c439a8..6cbfc613b1149 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalExpand.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalExpand.scala @@ -20,7 +20,8 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.calcite.Expand import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExpand -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -49,10 +50,10 @@ class StreamPhysicalExpand( override def translateToExecNode(): ExecNode[_] = { new StreamExecExpand( + unwrapTableConfig(this), projects, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGlobalGroupAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGlobalGroupAggregate.scala index 2e4dbb05f70fd..8f557037ddfc1 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGlobalGroupAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGlobalGroupAggregate.scala @@ -20,8 +20,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.PartialFinalType import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGlobalGroupAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -104,6 +105,7 @@ class StreamPhysicalGlobalGroupAggregate( override def translateToExecNode(): ExecNode[_] = { val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) new StreamExecGlobalGroupAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, aggCallNeedRetractions, @@ -113,7 +115,6 @@ class StreamPhysicalGlobalGroupAggregate( indexOfCountStar.map(Integer.valueOf).orNull, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGlobalWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGlobalWindowAggregate.scala index e6cffd41dbb08..f55c08fc27f3d 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGlobalWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGlobalWindowAggregate.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.rules.physical.stream.TwoStageOptimizedWindowAggregateRule import org.apache.flink.table.planner.plan.utils.WindowUtil.checkEmitConfiguration import org.apache.flink.table.planner.plan.utils.{AggregateUtil, FlinkRelOptUtil, RelExplainUtil, WindowUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -128,6 +129,7 @@ class StreamPhysicalGlobalWindowAggregate( override def translateToExecNode(): ExecNode[_] = { checkEmitConfiguration(FlinkRelOptUtil.getTableConfigFromContext(this)) new StreamExecGlobalWindowAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, windowing, @@ -135,7 +137,6 @@ class StreamPhysicalGlobalWindowAggregate( InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(inputRowTypeOfLocalAgg), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupAggregate.scala index 07bca07984748..c80ea2efa7e22 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupAggregate.scala @@ -20,8 +20,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.PartialFinalType import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{AggregateUtil, ChangelogPlanUtils, RelExplainUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -84,6 +85,7 @@ class StreamPhysicalGroupAggregate( val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecGroupAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, aggCallNeedRetractions, @@ -91,7 +93,6 @@ class StreamPhysicalGroupAggregate( needRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupTableAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupTableAggregate.scala index 5097a404a42ca..a94868b54c97b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupTableAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupTableAggregate.scala @@ -19,8 +19,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupTableAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{AggregateUtil, ChangelogPlanUtils} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -63,6 +64,7 @@ class StreamPhysicalGroupTableAggregate( val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecGroupTableAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, aggCallNeedRetractions, @@ -70,7 +72,6 @@ class StreamPhysicalGroupTableAggregate( needRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupWindowAggregate.scala index 679fc26fa273b..a24b8817ad757 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupWindowAggregate.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.logical._ import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupWindowAggregate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{ChangelogPlanUtils, WindowEmitStrategy} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -75,6 +76,7 @@ class StreamPhysicalGroupWindowAggregate( override def translateToExecNode(): ExecNode[_] = { val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecGroupWindowAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, window, @@ -82,7 +84,6 @@ class StreamPhysicalGroupWindowAggregate( needRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupWindowTableAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupWindowTableAggregate.scala index bdee5f655d06b..cc073cb2ebb5b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupWindowTableAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupWindowTableAggregate.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.planner.plan.logical._ import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupWindowAggregate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{ChangelogPlanUtils, WindowEmitStrategy} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -75,6 +76,7 @@ class StreamPhysicalGroupWindowTableAggregate( override def translateToExecNode(): ExecNode[_] = { val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecGroupWindowAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, window, @@ -82,7 +84,6 @@ class StreamPhysicalGroupWindowTableAggregate( needRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIncrementalGroupAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIncrementalGroupAggregate.scala index 48a2c999208a5..be37d1fef3931 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIncrementalGroupAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIncrementalGroupAggregate.scala @@ -19,8 +19,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecIncrementalGroupAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -118,6 +119,7 @@ class StreamPhysicalIncrementalGroupAggregate( override def translateToExecNode(): ExecNode[_] = { new StreamExecIncrementalGroupAggregate( + unwrapTableConfig(this), partialAggGrouping, finalAggGrouping, partialOriginalAggCalls, @@ -126,7 +128,6 @@ class StreamPhysicalIncrementalGroupAggregate( partialAggNeedRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIntervalJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIntervalJoin.scala index 9b26cd447e65c..dd4640d597361 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIntervalJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIntervalJoin.scala @@ -27,6 +27,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin import org.apache.flink.table.planner.plan.utils.PythonUtil.containsPythonCall import org.apache.flink.table.planner.plan.utils.RelExplainUtil.preferExpressionFormat +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan._ import org.apache.calcite.rel.core.{Join, JoinRelType} @@ -92,10 +93,11 @@ class StreamPhysicalIntervalJoin( override def translateToExecNode(): ExecNode[_] = { new StreamExecIntervalJoin( - new IntervalJoinSpec(joinSpec, windowBounds), - InputProperty.DEFAULT, - InputProperty.DEFAULT, - FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription) + unwrapTableConfig(this), + new IntervalJoinSpec(joinSpec, windowBounds), + InputProperty.DEFAULT, + InputProperty.DEFAULT, + FlinkTypeFactory.toLogicalRowType(getRowType), + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalJoin.scala index f4b68a9fde8cf..0d33d411774c6 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalJoin.scala @@ -20,10 +20,11 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery -import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecJoin +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin import org.apache.flink.table.planner.plan.utils.JoinUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.typeutils.InternalTypeInfo import org.apache.calcite.plan._ @@ -126,12 +127,13 @@ class StreamPhysicalJoin( override def translateToExecNode(): ExecNode[_] = { new StreamExecJoin( - joinSpec, - getUniqueKeys(left, joinSpec.getLeftKeys), - getUniqueKeys(right, joinSpec.getRightKeys), - InputProperty.DEFAULT, - InputProperty.DEFAULT, - FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription) + unwrapTableConfig(this), + joinSpec, + getUniqueKeys(left, joinSpec.getLeftKeys), + getUniqueKeys(right, joinSpec.getRightKeys), + InputProperty.DEFAULT, + InputProperty.DEFAULT, + FlinkTypeFactory.toLogicalRowType(getRowType), + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacySink.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacySink.scala index 4cb7bb7935d51..bdc2ff017b298 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacySink.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacySink.scala @@ -22,6 +22,7 @@ import org.apache.flink.table.planner.plan.nodes.calcite.LegacySink import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLegacySink import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{ChangelogPlanUtils, UpdatingPlanChecker} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType import org.apache.flink.table.sinks._ @@ -61,12 +62,12 @@ class StreamPhysicalLegacySink[T]( val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecLegacySink( + unwrapTableConfig(this), sink, upsertKeys.orNull, needRetraction, InputProperty.DEFAULT, fromDataTypeToLogicalType(sink.getConsumedDataType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacyTableSourceScan.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacyTableSourceScan.scala index 39ae30021508e..ae71c1f470c61 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacyTableSourceScan.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacyTableSourceScan.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNode import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLegacyTableSourceScan import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalLegacyTableSourceScan import org.apache.flink.table.planner.plan.schema.LegacyTableSourceTable +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.sources.StreamTableSource import org.apache.calcite.plan._ @@ -57,6 +58,7 @@ class StreamPhysicalLegacyTableSourceScan( override def translateToExecNode(): ExecNode[_] = { new StreamExecLegacyTableSourceScan( + unwrapTableConfig(this), tableSource, getTable.getQualifiedName, FlinkTypeFactory.toLogicalRowType(getRowType), diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLimit.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLimit.scala index 0e2bd82015bc0..264aaaff5dd6a 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLimit.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLimit.scala @@ -19,8 +19,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLimit -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{ChangelogPlanUtils, RelExplainUtil, SortUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel._ @@ -71,13 +72,13 @@ class StreamPhysicalLimit( val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecLimit( - limitStart, - limitEnd, - generateUpdateBefore, - needRetraction, - InputProperty.DEFAULT, - FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + unwrapTableConfig(this), + limitStart, + limitEnd, + generateUpdateBefore, + needRetraction, + InputProperty.DEFAULT, + FlinkTypeFactory.toLogicalRowType(getRowType), + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLocalGroupAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLocalGroupAggregate.scala index be2d352d903b7..684274ea5aad5 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLocalGroupAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLocalGroupAggregate.scala @@ -20,8 +20,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.PartialFinalType import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLocalGroupAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -91,13 +92,13 @@ class StreamPhysicalLocalGroupAggregate( override def translateToExecNode(): ExecNode[_] = { new StreamExecLocalGroupAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, aggCallNeedRetractions, needRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLocalWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLocalWindowAggregate.scala index 41fd057f5257f..7e5993875fcf2 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLocalWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLocalWindowAggregate.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.rules.physical.stream.TwoStageOptimizedWindowAggregateRule import org.apache.flink.table.planner.plan.utils.WindowUtil.checkEmitConfiguration import org.apache.flink.table.planner.plan.utils.{AggregateUtil, FlinkRelOptUtil, RelExplainUtil, WindowUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.{NamedWindowProperty, SliceEnd, WindowReference} import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -125,12 +126,12 @@ class StreamPhysicalLocalWindowAggregate( override def translateToExecNode(): ExecNode[_] = { checkEmitConfiguration(FlinkRelOptUtil.getTableConfigFromContext(this)) new StreamExecLocalWindowAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, windowing, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLookupJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLookupJoin.scala index f962e2909b0e8..d39c77795795c 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLookupJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLookupJoin.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalLookupJoin import org.apache.flink.table.planner.plan.utils.{FlinkRexUtil, JoinTypeUtil} import org.apache.flink.table.planner.utils.JavaScalaConversionUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptTable, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -77,6 +78,7 @@ class StreamPhysicalLookupJoin( (null, null) } new StreamExecLookupJoin( + unwrapTableConfig(this), JoinTypeUtil.getFlinkJoinType(joinType), remainingCondition.orNull, new TemporalTableSourceSpec(temporalTable), diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMatch.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMatch.scala index 252af7c0a9254..c449762da57ac 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMatch.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMatch.scala @@ -22,10 +22,11 @@ import org.apache.flink.table.api.TableException import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.logical.MatchRecognize import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecMatch -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.MatchUtil import org.apache.flink.table.planner.plan.utils.PythonUtil.containsPythonCall import org.apache.flink.table.planner.plan.utils.RelExplainUtil._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel._ @@ -97,10 +98,10 @@ class StreamPhysicalMatch( override def translateToExecNode(): ExecNode[_] = { new StreamExecMatch( + unwrapTableConfig(this), MatchUtil.createMatchSpec(logicalMatch), InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMiniBatchAssigner.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMiniBatchAssigner.scala index 524a0c59f41d1..d2e88acb38674 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMiniBatchAssigner.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMiniBatchAssigner.scala @@ -21,7 +21,8 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.`trait`.MiniBatchIntervalTraitDef import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecMiniBatchAssigner -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel} @@ -63,10 +64,10 @@ class StreamPhysicalMiniBatchAssigner( override def translateToExecNode(): ExecNode[_] = { val miniBatchInterval = traits.getTrait(MiniBatchIntervalTraitDef.INSTANCE).getMiniBatchInterval new StreamExecMiniBatchAssigner( + unwrapTableConfig(this), miniBatchInterval, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalOverAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalOverAggregate.scala index ff1ea7654d2cd..3409e3380127c 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalOverAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalOverAggregate.scala @@ -19,8 +19,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecOverAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.OverAggregateUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -57,10 +58,10 @@ class StreamPhysicalOverAggregate( override def translateToExecNode(): ExecNode[_] = { new StreamExecOverAggregate( + unwrapTableConfig(this), OverAggregateUtil.createOverSpec(logicWindow), InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonCalc.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonCalc.scala index b7eff1171a612..d946f708becc0 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonCalc.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonCalc.scala @@ -21,7 +21,8 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.api.TableException import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonCalc -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -58,6 +59,7 @@ class StreamPhysicalPythonCalc( } new StreamExecPythonCalc( + unwrapTableConfig(this), projection, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonCorrelate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonCorrelate.scala index 434f163a37b80..6c91710a95f3d 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonCorrelate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonCorrelate.scala @@ -20,9 +20,10 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.api.TableException import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonCorrelate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableFunctionScan import org.apache.flink.table.planner.plan.utils.JoinTypeUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -74,11 +75,11 @@ class StreamPhysicalPythonCorrelate( } new StreamExecPythonCorrelate( + unwrapTableConfig(this), JoinTypeUtil.getFlinkJoinType(joinType), scan.getCall.asInstanceOf[RexCall], InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupAggregate.scala index 6099021572373..53fca11f65f27 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupAggregate.scala @@ -20,8 +20,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonGroupAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -79,6 +80,7 @@ class StreamPhysicalPythonGroupAggregate( val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecPythonGroupAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, aggCallNeedRetractions, @@ -86,7 +88,6 @@ class StreamPhysicalPythonGroupAggregate( needRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupTableAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupTableAggregate.scala index c77a5d766c963..bf6afa3c79676 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupTableAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupTableAggregate.scala @@ -19,8 +19,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonGroupTableAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{AggregateUtil, ChangelogPlanUtils} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -63,6 +64,7 @@ class StreamPhysicalPythonGroupTableAggregate( val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecPythonGroupTableAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, aggCallNeedRetractions, @@ -70,7 +72,6 @@ class StreamPhysicalPythonGroupTableAggregate( needRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupWindowAggregate.scala index 3dc7043c7b1cd..ec623dee2f8bb 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupWindowAggregate.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.logical.LogicalWindow import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonGroupWindowAggregate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{ChangelogPlanUtils, WindowEmitStrategy} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -71,6 +72,7 @@ class StreamPhysicalPythonGroupWindowAggregate( val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecPythonGroupWindowAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, window, @@ -79,7 +81,6 @@ class StreamPhysicalPythonGroupWindowAggregate( needRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonOverAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonOverAggregate.scala index 677644b691a90..7f62f9f0bf90f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonOverAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonOverAggregate.scala @@ -19,8 +19,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonOverAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.OverAggregateUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -56,11 +57,11 @@ class StreamPhysicalPythonOverAggregate( override def translateToExecNode(): ExecNode[_] = { new StreamExecPythonOverAggregate( + unwrapTableConfig(this), OverAggregateUtil.createOverSpec(logicWindow), InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalRank.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalRank.scala index 61879f5bbe4ea..77e536bdf7409 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalRank.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalRank.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.spec.PartitionSpec import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecRank import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.operators.rank._ import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -106,6 +107,7 @@ class StreamPhysicalRank( val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) val fieldCollations = orderKey.getFieldCollations new StreamExecRank( + unwrapTableConfig(this), rankType, new PartitionSpec(partitionKey.toArray), SortUtil.getSortSpec(fieldCollations), @@ -115,7 +117,6 @@ class StreamPhysicalRank( generateUpdateBefore, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSink.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSink.scala index fcc391442b5e5..af63d5be3c4e0 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSink.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSink.scala @@ -27,6 +27,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.spec.DynamicTableSinkSpec import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{ChangelogPlanUtils, RelDescriptionWriterImpl} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -85,13 +86,13 @@ class StreamPhysicalSink( util.Arrays.asList(abilitySpecs: _*)) tableSinkSpec.setTableSink(tableSink) new StreamExecSink( + unwrapTableConfig(this), tableSinkSpec, inputChangelogMode, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), upsertMaterialize, - getDescriptionWithUpsert(upsertMaterialize) - ) + getDescriptionWithUpsert(upsertMaterialize)) } /** diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSort.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSort.scala index d0df328623d39..47331fa27cccd 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSort.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSort.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSort import org.apache.flink.table.planner.plan.utils.{RelExplainUtil, SortUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel._ @@ -71,11 +72,11 @@ class StreamPhysicalSort( override def translateToExecNode(): ExecNode[_] = { new StreamExecSort( + unwrapTableConfig(this), SortUtil.getSortSpec(sortCollation.getFieldCollations), InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSortLimit.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSortLimit.scala index 346a77b5e72be..0e1d45ade0196 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSortLimit.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSortLimit.scala @@ -19,8 +19,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSortLimit -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.core.Sort @@ -90,6 +91,7 @@ class StreamPhysicalSortLimit( override def translateToExecNode(): ExecNode[_] = { val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) new StreamExecSortLimit( + unwrapTableConfig(this), SortUtil.getSortSpec(sortCollation.getFieldCollations), limitStart, limitEnd, @@ -97,7 +99,6 @@ class StreamPhysicalSortLimit( generateUpdateBefore, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTableSourceScan.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTableSourceScan.scala index c334be817e4bc..a82160be5495c 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTableSourceScan.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTableSourceScan.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.spec.DynamicTableSourceSpe import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalTableSourceScan import org.apache.flink.table.planner.plan.schema.TableSourceTable +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan._ import org.apache.calcite.rel.RelNode @@ -63,6 +64,7 @@ class StreamPhysicalTableSourceScan( tableSourceSpec.setTableSource(tableSource) new StreamExecTableSourceScan( + unwrapTableConfig(this), tableSourceSpec, FlinkTypeFactory.toLogicalRowType(getRowType), getRelDetailedDescription) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTemporalJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTemporalJoin.scala index 4e1b359b4532d..933180bce029f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTemporalJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTemporalJoin.scala @@ -26,6 +26,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin import org.apache.flink.table.planner.plan.utils.TemporalJoinUtil import org.apache.flink.table.planner.plan.utils.TemporalJoinUtil.{TEMPORAL_JOIN_CONDITION, TEMPORAL_JOIN_CONDITION_PRIMARY_KEY} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.util.Preconditions.checkState import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -115,6 +116,7 @@ class StreamPhysicalTemporalJoin( } new StreamExecTemporalJoin( + unwrapTableConfig(this), temporalJoinSpec, isTemporalFunctionJoin, leftTimeAttributeInputRef, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTemporalSort.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTemporalSort.scala index 1e21455b41020..e1709d6aad1c0 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTemporalSort.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTemporalSort.scala @@ -22,6 +22,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTemporalSort import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{RelExplainUtil, SortUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel._ @@ -62,10 +63,10 @@ class StreamPhysicalTemporalSort( override def translateToExecNode(): ExecNode[_] = { new StreamExecTemporalSort( + unwrapTableConfig(this), SortUtil.getSortSpec(sortCollation.getFieldCollations), InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalUnion.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalUnion.scala index f45873573f7ae..fbc4a9611b8d1 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalUnion.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalUnion.scala @@ -20,7 +20,8 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecUnion -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -59,9 +60,9 @@ class StreamPhysicalUnion( override def translateToExecNode(): ExecNode[_] = { new StreamExecUnion( + unwrapTableConfig(this), getInputs.map(_ => InputProperty.DEFAULT), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalValues.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalValues.scala index 9b511a84845aa..07ed842d36326 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalValues.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalValues.scala @@ -21,6 +21,7 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.ExecNode import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecValues +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import com.google.common.collect.ImmutableList import org.apache.calcite.plan._ @@ -52,9 +53,9 @@ class StreamPhysicalValues( override def translateToExecNode(): ExecNode[_] = { new StreamExecValues( + unwrapTableConfig(this), tuples.asList().map(_.asList()), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWatermarkAssigner.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWatermarkAssigner.scala index 6ccbecb436b0f..46fe743103d9b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWatermarkAssigner.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWatermarkAssigner.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.nodes.calcite.WatermarkAssigner import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWatermarkAssigner import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.RelExplainUtil.preferExpressionFormat +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.{RelNode, RelWriter} @@ -70,6 +71,7 @@ class StreamPhysicalWatermarkAssigner( override def translateToExecNode(): ExecNode[_] = { new StreamExecWatermarkAssigner( + unwrapTableConfig(this), watermarkExpr, rowtimeFieldIndex, InputProperty.DEFAULT, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowAggregate.scala index eba1a5da1ebeb..8bdc707c7a454 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowAggregate.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowAgg import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.WindowUtil.checkEmitConfiguration import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -103,13 +104,13 @@ class StreamPhysicalWindowAggregate( override def translateToExecNode(): ExecNode[_] = { checkEmitConfiguration(FlinkRelOptUtil.getTableConfigFromContext(this)) new StreamExecWindowAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, windowing, namedWindowProperties.toArray, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowDeduplicate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowDeduplicate.scala index 1b77b2d8ad264..1a80564f76656 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowDeduplicate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowDeduplicate.scala @@ -22,6 +22,7 @@ import org.apache.flink.table.planner.plan.logical.WindowingStrategy import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowDeduplicate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel._ @@ -79,13 +80,13 @@ class StreamPhysicalWindowDeduplicate( override def translateToExecNode(): ExecNode[_] = { new StreamExecWindowDeduplicate( + unwrapTableConfig(this), partitionKeys, orderKey, keepLastRow, windowing, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowJoin.scala index cd3af33e5aad5..d24f806a05ac4 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowJoin.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin import org.apache.flink.table.planner.plan.utils.PythonUtil.containsPythonCall import org.apache.flink.table.planner.plan.utils.RelExplainUtil.preferExpressionFormat +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel._ @@ -110,6 +111,7 @@ class StreamPhysicalWindowJoin( override def translateToExecNode(): ExecNode[_] = { new StreamExecWindowJoin( + unwrapTableConfig(this), joinSpec, leftWindowing, rightWindowing, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowRank.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowRank.scala index 16688e55f5e73..2aed05ad59eaa 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowRank.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowRank.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.spec.PartitionSpec import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowRank import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.operators.rank._ import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -94,6 +95,7 @@ class StreamPhysicalWindowRank( override def translateToExecNode(): ExecNode[_] = { val fieldCollations = orderKey.getFieldCollations new StreamExecWindowRank( + unwrapTableConfig(this), rankType, new PartitionSpec(partitionKey.toArray), SortUtil.getSortSpec(fieldCollations), @@ -102,7 +104,6 @@ class StreamPhysicalWindowRank( windowing, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowTableFunction.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowTableFunction.scala index a2e2de5d6e927..4f438413fc1b9 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowTableFunction.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowTableFunction.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.logical.TimeAttributeWindowingStrateg import org.apache.flink.table.planner.plan.nodes.common.CommonPhysicalWindowTableFunction import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowTableFunction import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -60,10 +61,10 @@ class StreamPhysicalWindowTableFunction( override def translateToExecNode(): ExecNode[_] = { new StreamExecWindowTableFunction( + unwrapTableConfig(this), windowing, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala index 5e3c42b789903..ba08b50963b3c 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala @@ -41,7 +41,7 @@ class BatchCommonSubGraphBasedOptimizer(planner: BatchPlanner) override protected def doOptimize(roots: Seq[RelNode]): Seq[RelNodeBlock] = { // build RelNodeBlock plan - val rootBlocks = RelNodeBlockPlanBuilder.buildRelNodeBlockPlan(roots, planner.getConfiguration) + val rootBlocks = RelNodeBlockPlanBuilder.buildRelNodeBlockPlan(roots, planner.getTableConfig) // optimize recursively RelNodeBlock rootBlocks.foreach(optimizeBlock) rootBlocks @@ -79,7 +79,7 @@ class BatchCommonSubGraphBasedOptimizer(planner: BatchPlanner) private def optimizeTree(relNode: RelNode): RelNode = { val config = planner.getTableConfig val programs = TableConfigUtils.getCalciteConfig(config).getBatchProgram - .getOrElse(FlinkBatchProgram.buildProgram(config.getConfiguration)) + .getOrElse(FlinkBatchProgram.buildProgram(config)) Preconditions.checkNotNull(programs) val context = relNode.getCluster.getPlanner.getContext.unwrap(classOf[FlinkContext]) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala index 6205d937536cd..a4498587769e7 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala @@ -49,17 +49,17 @@ class StreamCommonSubGraphBasedOptimizer(planner: StreamPlanner) extends CommonSubGraphBasedOptimizer { override protected def doOptimize(roots: Seq[RelNode]): Seq[RelNodeBlock] = { - val config = planner.getConfiguration + val tableConfig = planner.getTableConfig // build RelNodeBlock plan - val sinkBlocks = RelNodeBlockPlanBuilder.buildRelNodeBlockPlan(roots, config) + val sinkBlocks = RelNodeBlockPlanBuilder.buildRelNodeBlockPlan(roots, tableConfig) // infer trait properties for sink block sinkBlocks.foreach { sinkBlock => // don't require update before by default sinkBlock.setUpdateBeforeRequired(false) - val miniBatchInterval: MiniBatchInterval = if (config.get( + val miniBatchInterval: MiniBatchInterval = if (tableConfig.get( ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED)) { - val miniBatchLatency = config.get( + val miniBatchLatency = tableConfig.get( ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY).toMillis Preconditions.checkArgument(miniBatchLatency > 0, "MiniBatch Latency must be greater than 0 ms.", null) @@ -156,7 +156,7 @@ class StreamCommonSubGraphBasedOptimizer(planner: StreamPlanner) val config = planner.getTableConfig val calciteConfig = TableConfigUtils.getCalciteConfig(config) val programs = calciteConfig.getStreamProgram - .getOrElse(FlinkStreamProgram.buildProgram(config.getConfiguration)) + .getOrElse(FlinkStreamProgram.buildProgram(config)) Preconditions.checkNotNull(programs) val context = relNode.getCluster.getPlanner.getContext.unwrap(classOf[FlinkContext]) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkBatchProgram.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkBatchProgram.scala index 7b351eeae5de8..df2c6e32019a7 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkBatchProgram.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkBatchProgram.scala @@ -18,7 +18,7 @@ package org.apache.flink.table.planner.plan.optimize.program -import org.apache.flink.configuration.Configuration +import org.apache.flink.configuration.ReadableConfig import org.apache.flink.table.api.config.OptimizerConfigOptions import org.apache.flink.table.planner.plan.nodes.FlinkConventions import org.apache.flink.table.planner.plan.rules.FlinkBatchRuleSets @@ -44,7 +44,7 @@ object FlinkBatchProgram { val PHYSICAL = "physical" val PHYSICAL_REWRITE = "physical_rewrite" - def buildProgram(config: Configuration): FlinkChainedProgram[BatchOptimizeContext] = { + def buildProgram(config: ReadableConfig): FlinkChainedProgram[BatchOptimizeContext] = { val chainedProgram = new FlinkChainedProgram[BatchOptimizeContext]() chainedProgram.addLast( @@ -151,7 +151,7 @@ object FlinkBatchProgram { .build()) // join reorder - if (config.getBoolean(OptimizerConfigOptions.TABLE_OPTIMIZER_JOIN_REORDER_ENABLED)) { + if (config.get(OptimizerConfigOptions.TABLE_OPTIMIZER_JOIN_REORDER_ENABLED)) { chainedProgram.addLast( JOIN_REORDER, FlinkGroupProgramBuilder.newBuilder[BatchOptimizeContext] diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala index 422c67df87c93..e34cc73bc4151 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala @@ -832,7 +832,7 @@ class FlinkChangelogModeInferenceProgram extends FlinkOptimizeProgram[StreamOpti val inputChangelogMode = ChangelogPlanUtils.getChangelogMode( sink.getInput.asInstanceOf[StreamPhysicalRel]).get val primaryKeys = sink.contextResolvedTable.getResolvedSchema.getPrimaryKeyIndexes - val upsertMaterialize = tableConfig.getConfiguration.get( + val upsertMaterialize = tableConfig.get( ExecutionConfigOptions.TABLE_EXEC_SINK_UPSERT_MATERIALIZE) match { case UpsertMaterialize.FORCE => primaryKeys.nonEmpty case UpsertMaterialize.NONE => false diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkStreamProgram.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkStreamProgram.scala index fa5f65c326352..c8a3fc50a23f1 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkStreamProgram.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkStreamProgram.scala @@ -18,7 +18,7 @@ package org.apache.flink.table.planner.plan.optimize.program -import org.apache.flink.configuration.Configuration +import org.apache.flink.configuration.ReadableConfig import org.apache.flink.table.api.config.OptimizerConfigOptions import org.apache.flink.table.planner.plan.nodes.FlinkConventions import org.apache.flink.table.planner.plan.rules.FlinkStreamRuleSets @@ -43,7 +43,7 @@ object FlinkStreamProgram { val PHYSICAL = "physical" val PHYSICAL_REWRITE = "physical_rewrite" - def buildProgram(config: Configuration): FlinkChainedProgram[StreamOptimizeContext] = { + def buildProgram(config: ReadableConfig): FlinkChainedProgram[StreamOptimizeContext] = { val chainedProgram = new FlinkChainedProgram[StreamOptimizeContext]() // rewrite sub-queries to joins @@ -161,7 +161,7 @@ object FlinkStreamProgram { .build()) // join reorder - if (config.getBoolean(OptimizerConfigOptions.TABLE_OPTIMIZER_JOIN_REORDER_ENABLED)) { + if (config.get(OptimizerConfigOptions.TABLE_OPTIMIZER_JOIN_REORDER_ENABLED)) { chainedProgram.addLast( JOIN_REORDER, FlinkGroupProgramBuilder.newBuilder[StreamOptimizeContext] diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/reuse/SubplanReuser.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/reuse/SubplanReuser.scala index 4bb1f27359b04..763f5b5d13940 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/reuse/SubplanReuser.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/reuse/SubplanReuser.scala @@ -56,11 +56,10 @@ object SubplanReuser { * Finds duplicated sub-plans and return the reused plan. */ def reuseDuplicatedSubplan(rels: Seq[RelNode], tableConfig: TableConfig): Seq[RelNode] = { - if (!tableConfig.getConfiguration.getBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED)) { + if (!tableConfig.get(OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED)) { return rels } - val tableSourceReuseEnabled = tableConfig.getConfiguration.getBoolean( + val tableSourceReuseEnabled = tableConfig.get( OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED) val context = new SubplanReuseContext(tableSourceReuseEnabled, rels: _*) val reuseShuttle = new SubplanReuseShuttle(context) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRule.scala index c47443430d452..96980f6142c35 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRule.scala @@ -26,7 +26,7 @@ import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelOptUtil} import org.apache.calcite.rel.core.Filter import org.apache.calcite.rex.{RexCall, RexLiteral, RexNode} import org.apache.calcite.sql.SqlBinaryOperator -import org.apache.calcite.sql.fun.SqlStdOperatorTable.{AND, EQUALS, IN, NOT_EQUALS, NOT_IN, OR} +import org.apache.calcite.sql.fun.SqlStdOperatorTable.{AND, EQUALS, IN, NOT, NOT_EQUALS, NOT_IN, OR} import org.apache.calcite.tools.RelBuilder import scala.collection.JavaConversions._ @@ -154,7 +154,9 @@ class ConvertToNotInOrInRule val values = list.map(_.getOperands.last) val call = toOperator match { case IN => builder.getRexBuilder.makeIn(inputRef, values) - case _ => builder.getRexBuilder.makeCall(toOperator, List(inputRef) ++ values) + case NOT_IN => builder + .getRexBuilder + .makeCall(NOT, builder.getRexBuilder.makeIn(inputRef, values)) } rexBuffer += call beenConverted = true diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/CorrelateSortToRankRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/CorrelateSortToRankRule.scala index b2d0797c7af27..e8013a8495965 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/CorrelateSortToRankRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/CorrelateSortToRankRule.scala @@ -175,7 +175,7 @@ class CorrelateSortToRankRule extends RelOptRule( 1, sort.fetch.asInstanceOf[RexLiteral].getValueAs(classOf[java.lang.Long])), null, - outputRankNumber = false) + false) .project(projects) .build() diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRule.scala index 099a1476b1479..069ffa8b25bd6 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRule.scala @@ -62,7 +62,7 @@ class JoinDeriveNullFilterRule val rexBuilder = join.getCluster.getRexBuilder val mq = FlinkRelMetadataQuery.reuseOrCreate(join.getCluster.getMetadataQuery) val conf = FlinkRelOptUtil.getTableConfigFromContext(join) - val minNullCount = conf.getConfiguration.getLong( + val minNullCount = conf.get( JoinDeriveNullFilterRule.TABLE_OPTIMIZER_JOIN_NULL_FILTER_THRESHOLD) def createIsNotNullFilter(input: RelNode, keys: ImmutableIntList): RelNode = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoLegacyTableSourceScanRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoLegacyTableSourceScanRule.scala index b1b5270a54995..25de4e5f9bb4f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoLegacyTableSourceScanRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoLegacyTableSourceScanRule.scala @@ -49,7 +49,7 @@ class PushFilterIntoLegacyTableSourceScanRule extends RelOptRule( override def matches(call: RelOptRuleCall): Boolean = { val config = call.getPlanner.getContext.unwrap(classOf[FlinkContext]).getTableConfig - if (!config.getConfiguration.getBoolean( + if (!config.get( OptimizerConfigOptions.TABLE_OPTIMIZER_SOURCE_PREDICATE_PUSHDOWN_ENABLED)) { return false } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRule.scala index 31d1f2527e7e8..3963a4b6cd6b4 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRule.scala @@ -128,7 +128,7 @@ class SplitAggregateRule extends RelOptRule( val tableConfig = call.getPlanner.getContext.unwrap(classOf[FlinkContext]).getTableConfig val agg: FlinkLogicalAggregate = call.rel(0) - val splitDistinctAggEnabled = tableConfig.getConfiguration.getBoolean( + val splitDistinctAggEnabled = tableConfig.get( OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED) val isAllAggSplittable = doAllAggSupportSplit(agg.getAggCallList) @@ -165,7 +165,7 @@ class SplitAggregateRule extends RelOptRule( }.distinct.diff(aggGroupSet).sorted.toArray val hashFieldsMap: util.Map[Int, Int] = new util.HashMap() - val buckets = tableConfig.getConfiguration.getInteger( + val buckets = tableConfig.get( OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_BUCKET_NUM) if (hashFieldIndexes.nonEmpty) { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalHashJoinRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalHashJoinRule.scala index f2b04ee5ee207..ea9232166e11b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalHashJoinRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalHashJoinRule.scala @@ -145,7 +145,7 @@ class BatchPhysicalHashJoinRule toHashTraitByColumns(joinInfo.rightKeys)) // add more possibility to only shuffle by partial joinKeys, now only single one - val isShuffleByPartialKeyEnabled = tableConfig.getConfiguration.getBoolean( + val isShuffleByPartialKeyEnabled = tableConfig.get( BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED) if (isShuffleByPartialKeyEnabled && joinInfo.pairs().length > 1) { joinInfo.pairs().foreach { pair => @@ -177,7 +177,7 @@ class BatchPhysicalHashJoinRule if (leftSize == null || rightSize == null) { return (false, false) } - val threshold = tableConfig.getConfiguration.getLong( + val threshold = tableConfig.get( OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD) joinType match { case JoinRelType.LEFT => (rightSize <= threshold, false) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalJoinRuleBase.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalJoinRuleBase.scala index 14c0886a3ca24..6e73cc9b00c79 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalJoinRuleBase.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalJoinRuleBase.scala @@ -56,7 +56,7 @@ trait BatchPhysicalJoinRuleBase { distinctKeys: Seq[Int]): Boolean = { val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(buildRel) val mq = buildRel.getCluster.getMetadataQuery - val ratioConf = tableConfig.getConfiguration.getDouble( + val ratioConf = tableConfig.get( BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SEMI_JOIN_BUILD_DISTINCT_NDV_RATIO) val inputRows = mq.getRowCount(buildRel) val ndvOfGroupKey = mq.getDistinctRowCount( diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSortMergeJoinRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSortMergeJoinRule.scala index 9a72afe217333..321ca264b9476 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSortMergeJoinRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSortMergeJoinRule.scala @@ -109,7 +109,7 @@ class BatchPhysicalSortMergeJoinRule } val tableConfig = call.getPlanner.getContext.unwrap(classOf[FlinkContext]).getTableConfig - val candidates = if (tableConfig.getConfiguration.getBoolean( + val candidates = if (tableConfig.get( BatchPhysicalSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED)) { // add more possibility to remove redundant sort, and longer optimization time Array((false, false), (true, false), (false, true), (true, true)) @@ -127,7 +127,7 @@ class BatchPhysicalSortMergeJoinRule } // add more possibility to only shuffle by partial joinKeys, now only single one - val isShuffleByPartialKeyEnabled = tableConfig.getConfiguration.getBoolean( + val isShuffleByPartialKeyEnabled = tableConfig.get( BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED) if (isShuffleByPartialKeyEnabled && joinInfo.pairs().length > 1) { joinInfo.pairs().foreach { pair => diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSortRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSortRule.scala index 6fc392b0a51b8..bbd85ccb36f63 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSortRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSortRule.scala @@ -53,7 +53,7 @@ class BatchPhysicalSortRule extends ConverterRule( val sort: FlinkLogicalSort = rel.asInstanceOf[FlinkLogicalSort] val input = sort.getInput val config = sort.getCluster.getPlanner.getContext.unwrap(classOf[FlinkContext]).getTableConfig - val enableRangeSort = config.getConfiguration.getBoolean( + val enableRangeSort = config.get( BatchPhysicalSortRule.TABLE_EXEC_RANGE_SORT_ENABLED) val distribution = if (enableRangeSort) { FlinkRelDistribution.range(sort.getCollation.getFieldCollations) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/IncrementalAggregateRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/IncrementalAggregateRule.scala index 5aa549cb2ac87..0e4ab86a791ae 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/IncrementalAggregateRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/IncrementalAggregateRule.scala @@ -55,7 +55,7 @@ class IncrementalAggregateRule val tableConfig = call.getPlanner.getContext.unwrap(classOf[FlinkContext]).getTableConfig // whether incremental aggregate is enabled - val incrementalAggEnabled = tableConfig.getConfiguration.getBoolean( + val incrementalAggEnabled = tableConfig.get( IncrementalAggregateRule.TABLE_OPTIMIZER_INCREMENTAL_AGG_ENABLED) partialGlobalAgg.partialFinalType == PartialFinalType.PARTIAL && diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala index fff97aceb133f..a8533e25bbf6a 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala @@ -64,8 +64,7 @@ class MiniBatchIntervalInferRule extends RelOptRule( val miniBatchIntervalTrait = rel.getTraitSet.getTrait(MiniBatchIntervalTraitDef.INSTANCE) val inputs = getInputs(rel) val config = FlinkRelOptUtil.getTableConfigFromContext(rel) - val miniBatchEnabled = config.getConfiguration.getBoolean( - ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED) + val miniBatchEnabled = config.get(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED) val updatedTrait = rel match { case _: StreamPhysicalGroupWindowAggregate => diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/TwoStageOptimizedAggregateRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/TwoStageOptimizedAggregateRule.scala index 1399bcfc3457e..e00b6ea101b24 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/TwoStageOptimizedAggregateRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/TwoStageOptimizedAggregateRule.scala @@ -76,8 +76,7 @@ class TwoStageOptimizedAggregateRule extends RelOptRule( needRetraction, isStateBackendDataViews = true) - val isMiniBatchEnabled = tableConfig.getConfiguration.getBoolean( - ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED) + val isMiniBatchEnabled = tableConfig.get(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED) val isTwoPhaseEnabled = getAggPhaseStrategy(tableConfig) != AggregatePhaseStrategy.ONE_PHASE isMiniBatchEnabled && isTwoPhaseEnabled && diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/schema/LegacyCatalogSourceTable.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/schema/LegacyCatalogSourceTable.scala index 0d38e5745ee9f..3d44df5ba7a48 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/schema/LegacyCatalogSourceTable.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/schema/LegacyCatalogSourceTable.scala @@ -81,11 +81,11 @@ class LegacyCatalogSourceTable[T]( .unwrap(classOf[FlinkContext]) val typeFactory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory] - val conf = flinkContext.getTableConfig.getConfiguration + val conf = flinkContext.getTableConfig val hintedOptions = FlinkHints.getHintedOptions(context.getTableHints) if (hintedOptions.nonEmpty - && !conf.getBoolean(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED)) { + && !conf.get(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED)) { throw new ValidationException(s"${FlinkHints.HINT_NAME_OPTIONS} hint is allowed only when " + s"${TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED.key} " + s"is set to true") diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala index 7aa7524228ae1..bf8f67e31e3f3 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala @@ -200,8 +200,7 @@ object FlinkRelOptUtil { /** Get max cnf node limit by context of rel */ def getMaxCnfNodeCount(rel: RelNode): Int = { - val tableConfig = getTableConfigFromContext(rel) - tableConfig.getConfiguration.getInteger(FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT) + getTableConfigFromContext(rel).get(FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT) } /** diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtil.scala index 6f78ac30d4fc8..6f40ad25b7e6e 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtil.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtil.scala @@ -20,7 +20,6 @@ package org.apache.flink.table.planner.plan.utils import org.apache.flink.annotation.Experimental import org.apache.flink.configuration.ConfigOption import org.apache.flink.configuration.ConfigOptions.key -import org.apache.flink.table.planner.JList import org.apache.flink.table.planner.functions.sql.SqlTryCastFunction import org.apache.flink.table.planner.plan.utils.ExpressionDetail.ExpressionDetail import org.apache.flink.table.planner.plan.utils.ExpressionFormat.ExpressionFormat @@ -35,7 +34,7 @@ import org.apache.calcite.sql.`type`.SqlTypeName import org.apache.calcite.sql.fun.SqlStdOperatorTable._ import org.apache.calcite.sql.fun.{SqlCastFunction, SqlStdOperatorTable} import org.apache.calcite.sql.{SqlAsOperator, SqlKind, SqlOperator} -import org.apache.calcite.util.{ControlFlowException, DateString, ImmutableBitSet, NlsString, Sarg, TimeString, TimestampString, Util} +import org.apache.calcite.util._ import java.lang.{Iterable => JIterable} import java.math.BigDecimal @@ -397,17 +396,6 @@ object FlinkRexUtil { rex.accept(shuttle) } - /** Expands the Sarg operands to literals. */ - def expandSearchOperands(rexBuilder: RexBuilder, call: RexCall): JList[RexNode] = { - require(call.getKind == SqlKind.SEARCH) - val sargLiteral = call.getOperands.get(1).asInstanceOf[RexLiteral] - val sarg = sargLiteral.getValueAs(classOf[Sarg[_]]) - require(sarg.isPoints) - val sargOperands = sarg.rangeSet.asRanges().map(range => - rexBuilder.makeLiteral(range.lowerEndpoint(), sargLiteral.getType, false)) - List(call.getOperands.head) ++ sargOperands - } - /** * Adjust the expression's field indices according to fieldsOldToNewIndexMapping. * diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/IntervalJoinUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/IntervalJoinUtil.scala index dbb60ff08d7eb..303690b26bfa6 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/IntervalJoinUtil.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/IntervalJoinUtil.scala @@ -74,8 +74,7 @@ object IntervalJoinUtil { // Converts the condition to conjunctive normal form (CNF) val cnfCondition = FlinkRexUtil.toCnf(rexBuilder, - tableConfig.getConfiguration.getInteger(FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT), - predicate) + tableConfig.get(FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT), predicate) // split the condition into time predicates and other predicates // We need two range predicates or an equality predicate for a properly bounded window join. diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/PartitionPruner.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/PartitionPruner.scala index 9f1c47b005f77..49d05b329324f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/PartitionPruner.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/PartitionPruner.scala @@ -119,13 +119,8 @@ object PartitionPruner { val results: JList[Boolean] = new JArrayList[Boolean](allPartitions.size) val collector = new ListCollector[Boolean](results) - val parameters = if (tableConfig.getConfiguration != null) { - tableConfig.getConfiguration - } else { - new Configuration() - } try { - richMapFunction.open(parameters) + richMapFunction.open(new Configuration) // do filter against all partitions allPartitions.foreach { partition => val row = convertPartitionToRow( diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RankUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RankUtil.scala index ae9e522d9c075..77785ff9bad1a 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RankUtil.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RankUtil.scala @@ -75,8 +75,7 @@ object RankUtil { tableConfig: TableConfig): (Option[RankRange], Option[RexNode]) = { val predicate = FlinkRexUtil.expandSearch(rexBuilder, oriPred) // Converts the condition to conjunctive normal form (CNF) - val cnfNodeCount = tableConfig.getConfiguration.getInteger( - FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT) + val cnfNodeCount = tableConfig.get(FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT) val cnfCondition = FlinkRexUtil.toCnf(rexBuilder, cnfNodeCount, predicate) // split the condition into sort limit condition and other condition diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/SetOpRewriteUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/SetOpRewriteUtil.scala index eb6d422711403..32e400f1a1b14 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/SetOpRewriteUtil.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/SetOpRewriteUtil.scala @@ -20,6 +20,7 @@ package org.apache.flink.table.planner.plan.utils import org.apache.flink.table.functions.BuiltInFunctionDefinitions +import org.apache.flink.table.planner.calcite.FlinkRelBuilder import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction import org.apache.calcite.plan.RelOptUtil @@ -73,12 +74,15 @@ object SetOpRewriteUtil { val cluster = relBuilder.getCluster val sqlFunction = BridgingSqlFunction.of( - relBuilder.getCluster, + cluster, BuiltInFunctionDefinitions.INTERNAL_REPLICATE_ROWS) - relBuilder - .functionScan(sqlFunction, 0, relBuilder.fields(Util.range(fields.size() + 1))) - .rename(outputRelDataType.getFieldNames) + FlinkRelBuilder.pushFunctionScan( + relBuilder, + sqlFunction, + 0, + relBuilder.fields(Util.range(fields.size() + 1)), + outputRelDataType.getFieldNames) // correlated join val corSet = Collections.singleton(cluster.createCorrel()) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowUtil.scala index 6a23c3b518764..69efd3473d416 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowUtil.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowUtil.scala @@ -254,9 +254,8 @@ object WindowUtil { * throws exception when the configurations are set. */ def checkEmitConfiguration(tableConfig: TableConfig): Unit = { - val conf = tableConfig.getConfiguration - if (conf.getBoolean(TABLE_EXEC_EMIT_EARLY_FIRE_ENABLED) || - conf.getBoolean(TABLE_EXEC_EMIT_LATE_FIRE_ENABLED)) { + if (tableConfig.get(TABLE_EXEC_EMIT_EARLY_FIRE_ENABLED) || + tableConfig.get(TABLE_EXEC_EMIT_LATE_FIRE_ENABLED)) { throw new TableException("Currently, window table function based aggregate doesn't " + s"support early-fire and late-fire configuration " + s"'${TABLE_EXEC_EMIT_EARLY_FIRE_ENABLED.key()}' and " + diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/file/table/LegacyTableFactory.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/file/table/LegacyTableFactory.java new file mode 100644 index 0000000000000..c0b03896c257d --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/file/table/LegacyTableFactory.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.file.table; + +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.descriptors.DescriptorProperties; +import org.apache.flink.table.factories.StreamTableSinkFactory; +import org.apache.flink.table.factories.StreamTableSourceFactory; +import org.apache.flink.table.factories.TableFactory; +import org.apache.flink.table.planner.runtime.utils.TestingAppendTableSink; +import org.apache.flink.table.planner.utils.TestTableSource; +import org.apache.flink.table.sinks.StreamTableSink; +import org.apache.flink.table.sources.StreamTableSource; +import org.apache.flink.types.Row; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.table.descriptors.Schema.SCHEMA; + +/** A legacy {@link TableFactory} uses user define options. */ +public class LegacyTableFactory + implements StreamTableSinkFactory, StreamTableSourceFactory { + + @Override + public StreamTableSink createStreamTableSink(Map properties) { + DescriptorProperties dp = new DescriptorProperties(); + dp.putProperties(properties); + TableSchema tableSchema = dp.getTableSchema(SCHEMA); + StreamTableSink sink = new TestingAppendTableSink(); + return (StreamTableSink) + sink.configure(tableSchema.getFieldNames(), tableSchema.getFieldTypes()); + } + + @Override + public StreamTableSource createStreamTableSource(Map properties) { + DescriptorProperties dp = new DescriptorProperties(); + dp.putProperties(properties); + TableSchema tableSchema = dp.getTableSchema(SCHEMA); + return new TestTableSource(false, tableSchema); + } + + @Override + public Map requiredContext() { + Map options = new HashMap<>(); + options.put("type", "legacy"); + return options; + } + + @Override + public List supportedProperties() { + List properties = new ArrayList<>(); + // schema + properties.add(SCHEMA + ".#." + DescriptorProperties.TYPE); + properties.add(SCHEMA + ".#." + DescriptorProperties.DATA_TYPE); + properties.add(SCHEMA + ".#." + DescriptorProperties.NAME); + properties.add(SCHEMA + ".#." + DescriptorProperties.EXPR); + return properties; + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/CompiledPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/CompiledPlanITCase.java index 27997b6e96545..d2ae077e13283 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/CompiledPlanITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/CompiledPlanITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.table.api; import org.apache.flink.FlinkVersion; +import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.api.internal.TableResultInternal; import org.apache.flink.table.planner.utils.JsonPlanTestBase; @@ -36,6 +37,8 @@ import java.nio.file.Paths; import java.util.Arrays; import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; import static org.apache.flink.table.api.Expressions.$; @@ -74,7 +77,7 @@ public void setup() throws Exception { @Test public void testCompilePlanSql() throws IOException { CompiledPlan compiledPlan = - tableEnv.compilePlanSql("insert into MySink select * from MyTable"); + tableEnv.compilePlanSql("INSERT INTO MySink SELECT * FROM MyTable"); String expected = TableTestUtil.readFromResource("/jsonplan/testGetJsonPlan.out"); assertThat( TableTestUtil.replaceExecNodeId( @@ -91,7 +94,7 @@ public void testCompilePlanSql() throws IOException { public void testExecutePlanSql() throws Exception { File sinkPath = createSourceSinkTables(); - tableEnv.compilePlanSql("insert into sink select * from src").execute().await(); + tableEnv.compilePlanSql("INSERT INTO sink SELECT * FROM src").execute().await(); assertResult(DATA, sinkPath); } @@ -112,7 +115,7 @@ public void testCompileWriteToFileAndThenExecuteSql() throws Exception { File sinkPath = createSourceSinkTables(); - CompiledPlan plan = tableEnv.compilePlanSql("insert into sink select * from src"); + CompiledPlan plan = tableEnv.compilePlanSql("INSERT INTO sink SELECT * FROM src"); plan.writeToFile(planPath); tableEnv.executeSql(String.format("EXECUTE PLAN '%s'", planPath.toAbsolutePath())).await(); @@ -315,6 +318,46 @@ public void testExplainPlan() throws IOException { .isEqualTo(expected); } + @Test + public void testPersistedConfigOption() throws Exception { + Path planPath = Paths.get(URI.create(getTempDirPath("plan")).getPath(), "plan.json"); + FileUtils.createParentDirectories(planPath.toFile()); + + List data = + Stream.concat( + DATA.stream(), + Stream.of( + "4,2,This string is long", + "5,3,This is an even longer string")) + .collect(Collectors.toList()); + String[] sinkColumnDefinitions = new String[] {"a bigint", "b int", "c varchar(11)"}; + + createTestCsvSourceTable("src", data, COLUMNS_DEFINITION); + File sinkPath = createTestCsvSinkTable("sink", sinkColumnDefinitions); + + // Set config option to trim the strings, so it's persisted in the json plan + tableEnv.getConfig() + .getConfiguration() + .set( + ExecutionConfigOptions.TABLE_EXEC_SINK_TYPE_LENGTH_ENFORCER, + ExecutionConfigOptions.TypeLengthEnforcer.TRIM_PAD); + CompiledPlan plan = tableEnv.compilePlanSql("INSERT INTO sink SELECT * FROM src"); + + // Set config option to trim the strings to IGNORE, to validate that the persisted config + // is overriding the environment setting. + tableEnv.getConfig() + .getConfiguration() + .set( + ExecutionConfigOptions.TABLE_EXEC_SINK_TYPE_LENGTH_ENFORCER, + ExecutionConfigOptions.TypeLengthEnforcer.IGNORE); + + plan.execute().await(); + List expected = + Stream.concat(DATA.stream(), Stream.of("4,2,This string", "5,3,This is an ")) + .collect(Collectors.toList()); + assertResult(expected, sinkPath); + } + @Test public void testBatchMode() { tableEnv = TableEnvironment.create(EnvironmentSettings.inBatchMode()); @@ -335,7 +378,7 @@ public void testBatchMode() { + " 'table-sink-class' = 'DEFAULT')"; tableEnv.executeSql(sinkTableDdl); - assertThatThrownBy(() -> tableEnv.compilePlanSql("insert into sink select * from src")) + assertThatThrownBy(() -> tableEnv.compilePlanSql("INSERT INTO sink SELECT * FROM src")) .isInstanceOf(UnsupportedOperationException.class) .hasMessage("The compiled plan feature is not supported in batch mode."); } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/EnvironmentTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/EnvironmentTest.java index ff349536723bb..eaa25823996df 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/EnvironmentTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/EnvironmentTest.java @@ -24,12 +24,15 @@ import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.types.Row; import org.junit.Test; import java.time.Duration; +import java.util.concurrent.ExecutionException; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; /** Tests for {@link TableEnvironment} that require a planner. */ @@ -61,4 +64,29 @@ public void testPassingExecutionParameters() { assertEquals(800, env.getConfig().getAutoWatermarkInterval()); assertEquals(30000, env.getCheckpointConfig().getCheckpointInterval()); } + + @Test + public void testEnvironmentSettings() throws ExecutionException, InterruptedException { + Configuration conf = new Configuration(); + conf.set(TableConfigOptions.TABLE_CATALOG_NAME, "myCatalog"); + EnvironmentSettings settings = + EnvironmentSettings.newInstance().withConfiguration(conf).build(); + + TableEnvironment tEnv = TableEnvironment.create(settings); + assertThat(tEnv.getConfig().get(TableConfigOptions.TABLE_CATALOG_NAME)) + .isEqualTo("myCatalog"); + assertThat(tEnv.getCurrentCatalog()).isEqualTo("myCatalog"); + + StreamTableEnvironment stEnv = + StreamTableEnvironment.create( + StreamExecutionEnvironment.getExecutionEnvironment(), settings); + assertThat(stEnv.getConfig().get(TableConfigOptions.TABLE_CATALOG_NAME)) + .isEqualTo("myCatalog"); + + stEnv.getConfig() + .set( + TableConfigOptions.TABLE_CATALOG_NAME, + TableConfigOptions.TABLE_CATALOG_NAME.defaultValue()); + assertThat(stEnv.getCurrentCatalog()).isEqualTo("myCatalog"); + } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java index db9585ea4bd4f..1c93fe10e022a 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java @@ -402,8 +402,7 @@ public DynamicTableSource createDynamicTableSource(Context context) { Collection data = registeredData.getOrDefault(dataId, Collections.emptyList()); List> partitions = parsePartitionList(helper.getOptions().get(PARTITION_LIST)); - DataType producedDataType = - context.getCatalogTable().getSchema().toPhysicalRowDataType(); + DataType producedDataType = context.getPhysicalRowDataType(); // pushing project into scan will prune schema and we have to get the mapping between // partition and row Map, Collection> partition2Rows; diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/BuiltInAggregateFunctionTestBase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/BuiltInAggregateFunctionTestBase.java index 352ed83c55b7c..fc65b91a67c2a 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/BuiltInAggregateFunctionTestBase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/BuiltInAggregateFunctionTestBase.java @@ -18,8 +18,7 @@ package org.apache.flink.table.planner.functions; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.StateChangelogOptions; +import org.apache.flink.configuration.StateBackendOptions; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.table.api.EnvironmentSettings; @@ -79,9 +78,7 @@ public class BuiltInAggregateFunctionTestBase { public void testFunction() throws Exception { final TableEnvironment tEnv = TableEnvironment.create(EnvironmentSettings.inStreamingMode()); - Configuration configuration = tEnv.getConfig().getConfiguration(); - // see https://issues.apache.org/jira/browse/FLINK-26092 - configuration.set(StateChangelogOptions.ENABLE_STATE_CHANGE_LOG, false); + tEnv.getConfig().set(StateBackendOptions.STATE_BACKEND, testSpec.backendName); final Table sourceTable = asTable(tEnv, testSpec.sourceRowType, testSpec.sourceRows); for (final TestItem testItem : testSpec.testItems) { @@ -177,6 +174,7 @@ protected static class TestSpec { private DataType sourceRowType; private List sourceRows; + private String backendName; private TestSpec(BuiltInFunctionDefinition definition) { this.definition = Preconditions.checkNotNull(definition); @@ -197,6 +195,15 @@ TestSpec withSource(DataType sourceRowType, List sourceRows) { return this; } + public TestSpec withStateBackend(String backendName) { + this.backendName = backendName; + return this; + } + + public TestSpec copy() { + return new TestSpec(definition).withSource(sourceRowType, new ArrayList<>(sourceRows)); + } + TestSpec testSqlResult( Function sqlSpec, DataType expectedRowType, List expectedRows) { this.testItems.add(new SqlTestItem(sqlSpec, expectedRowType, expectedRows)); @@ -240,6 +247,7 @@ public String toString() { bob.append(description); bob.append(")"); } + bob.append(", backend: ").append(backendName); return bob.toString(); } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionITCase.java index baa7d366419e8..86629098f035e 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionITCase.java @@ -188,11 +188,11 @@ public static List allTypesBasic() { .build(), CastTestSpecBuilder.testCastTo(BINARY(2)) .fromCase(BINARY(5), null, null) - .fromCase(CHAR(4), "666F", new byte[] {102, 111}) - .fromCase(VARCHAR(8), "666f", new byte[] {102, 111}) - .fromCase(STRING(), "AAbbcCdD", new byte[] {-86, -69}) - .fromCase(VARCHAR(4), "FC", new byte[] {-4, 0}) - .fromCase(STRING(), "df", new byte[] {-33, 0}) + .fromCase(CHAR(4), "666F", new byte[] {54, 54}) + .fromCase(VARCHAR(8), "666f", new byte[] {54, 54}) + .fromCase(STRING(), "a", new byte[] {97, 0}) + .fromCase(VARCHAR(4), "FC", new byte[] {70, 67}) + .fromCase(STRING(), "foobar", new byte[] {102, 111}) // Not supported - no fix .failValidation(BOOLEAN(), true) // @@ -226,9 +226,9 @@ public static List allTypesBasic() { .build(), CastTestSpecBuilder.testCastTo(VARBINARY(4)) .fromCase(VARBINARY(5), null, null) - .fromCase(CHAR(4), "666F", new byte[] {102, 111}) - .fromCase(VARCHAR(8), "666f", new byte[] {102, 111}) - .fromCase(STRING(), "AAbbCcDdEe", new byte[] {-86, -69, -52, -35}) + .fromCase(CHAR(4), "foo", new byte[] {102, 111, 111, 32}) + .fromCase(VARCHAR(8), "foobar", new byte[] {102, 111, 111, 98}) + .fromCase(STRING(), "AAbbCcDdEe", new byte[] {65, 65, 98, 98}) // Not supported - no fix .failValidation(BOOLEAN(), true) // @@ -260,9 +260,12 @@ public static List allTypesBasic() { .build(), CastTestSpecBuilder.testCastTo(BYTES()) .fromCase(BYTES(), null, null) - .fromCase(CHAR(4), "666f", new byte[] {102, 111}) - .fromCase(VARCHAR(8), "666F", new byte[] {102, 111}) - .fromCase(STRING(), "aaBBCcDdEe", new byte[] {-86, -69, -52, -35, -18}) + .fromCase(CHAR(4), "foo", new byte[] {102, 111, 111, 32}) + .fromCase(VARCHAR(8), "foobar", new byte[] {102, 111, 111, 98, 97, 114}) + .fromCase( + STRING(), + "Apache Flink", + new byte[] {65, 112, 97, 99, 104, 101, 32, 70, 108, 105, 110, 107}) // Not supported - no fix .failValidation(BOOLEAN(), true) // @@ -992,11 +995,11 @@ public static List toStringCasts() { .fromCase(STRING(), "Apache Flink", "Apache Flink") .fromCase(STRING(), null, null) .fromCase(BOOLEAN(), true, "TRUE") - .fromCase(BINARY(2), DEFAULT_BINARY, "0001") - .fromCase(BINARY(3), DEFAULT_BINARY, "000100") - .fromCase(VARBINARY(3), DEFAULT_VARBINARY, "000102") - .fromCase(VARBINARY(5), DEFAULT_VARBINARY, "000102") - .fromCase(BYTES(), DEFAULT_BYTES, "0001020304") + .fromCase(BINARY(2), DEFAULT_BINARY, "\u0000\u0001") + .fromCase(BINARY(3), DEFAULT_BINARY, "\u0000\u0001\u0000") + .fromCase(VARBINARY(3), DEFAULT_VARBINARY, "\u0000\u0001\u0002") + .fromCase(VARBINARY(5), DEFAULT_VARBINARY, "\u0000\u0001\u0002") + .fromCase(BYTES(), DEFAULT_BYTES, "\u0000\u0001\u0002\u0003\u0004") .fromCase(DECIMAL(4, 3), 9.87, "9.870") .fromCase(DECIMAL(10, 5), 1, "1.00000") .fromCase( diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionMiscITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionMiscITCase.java index 7d312468fe6a2..473dd7dc49885 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionMiscITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionMiscITCase.java @@ -206,14 +206,14 @@ public static List testData() { .onFieldsWithData("foo") .testSqlResult( "CAST(CAST(x'68656C6C6F20636F6465' AS BINARY(10)) AS VARCHAR)", - "68656c6c6f20636f6465", + "hello code", STRING().notNull()), TestSpec.forFunction( BuiltInFunctionDefinitions.CAST, "test the x'....' binary syntax") .onFieldsWithData("foo") .testSqlResult( - "CAST(CAST(x'68656C6C6F2063617374' AS BINARY(10)) AS VARCHAR)", - "68656c6c6f2063617374", + "CAST(CAST(x'68656C6C6F20636F6465' AS BINARY(5)) AS VARCHAR)", + "hello", STRING().notNull()), TestSpec.forFunction(BuiltInFunctionDefinitions.CAST, "cast STRUCTURED to STRING") .onFieldsWithData(123456, "Flink") diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/GreatestLeastFunctionsITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/GreatestLeastFunctionsITCase.java index 178043841748d..d495c3f98e1d1 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/GreatestLeastFunctionsITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/GreatestLeastFunctionsITCase.java @@ -81,20 +81,63 @@ public static List testData() { call("GREATEST", $("f6"), $("f7")), "GREATEST(f6, f7)", LocalDateTime.parse("1970-01-01T00:00:03.001"), - DataTypes.TIMESTAMP(3).notNull())) + DataTypes.TIMESTAMP(3).notNull()), + resultSpec( + call("GREATEST", $("f0"), $("f1"), $("f2")), + "GREATEST(f0, f1, f2)", + null, + DataTypes.INT()), + resultSpec( + call("GREATEST", $("f4"), $("f5")), + "GREATEST(f4, f5)", + "world", + DataTypes.STRING().notNull()), + resultSpec( + call("GREATEST", $("f6"), $("f7")), + "GREATEST(f6, f7)", + LocalDateTime.parse("1970-01-01T00:00:03.001"), + DataTypes.TIMESTAMP(3).notNull()), + // assert that primitive types are returned and used in the equality + // operator applied on top of the GREATEST functions + resultSpec( + call( + "EQUALS", + call("GREATEST", $("f1"), $("f2")), + call("GREATEST", $("f1"), $("f2"))), + "GREATEST(f1, f2) = GREATEST(f1, f2)", + true, + DataTypes.BOOLEAN().notNull()), + resultSpec( + call( + "EQUALS", + call("GREATEST", $("f0"), $("f1")), + call("GREATEST", $("f0"), $("f1"))), + "GREATEST(f0, f1) = GREATEST(f0, f1)", + null, + DataTypes.BOOLEAN())) .testSqlValidationError( "GREATEST(f5, f6)", "SQL validation failed. Invalid function call:\n" + "GREATEST(STRING NOT NULL, TIMESTAMP(3) NOT NULL)"), TestSpec.forFunction(BuiltInFunctionDefinitions.LEAST) - .onFieldsWithData(null, 1, 2, 3.14, "hello", "world") + .onFieldsWithData( + null, + 1, + 2, + 3.14, + "hello", + "world", + LocalDateTime.parse("1970-01-01T00:00:03.001"), + LocalDateTime.parse("1970-01-01T00:00:02.001")) .andDataTypes( DataTypes.INT().nullable(), DataTypes.INT().notNull(), DataTypes.INT().notNull(), DataTypes.DECIMAL(3, 2).notNull(), DataTypes.STRING().notNull(), - DataTypes.STRING().notNull()) + DataTypes.STRING().notNull(), + DataTypes.TIMESTAMP(3).notNull(), + DataTypes.TIMESTAMP(3).notNull()) .testSqlValidationError( "LEAST(f1, f4)", "SQL validation failed. Invalid function call:\n" @@ -115,6 +158,38 @@ public static List testData() { call("LEAST", $("f4"), $("f5")), "LEAST(f4, f5)", "hello", - DataTypes.STRING().notNull()))); + DataTypes.STRING().notNull()), + resultSpec( + call("LEAST", $("f0"), $("f1")), + "LEAST(f0, f1)", + null, + DataTypes.INT()), + resultSpec( + call("LEAST", $("f4"), $("f5")), + "LEAST(f4, f5)", + "hello", + DataTypes.STRING().notNull()), + // assert that primitive types are returned and used in the equality + // operator applied on top of the GREATEST functions + resultSpec( + call( + "EQUALS", + call("LEAST", $("f1"), $("f2")), + call("LEAST", $("f1"), $("f2"))), + "LEAST(f1, f2) = LEAST(f1, f2)", + true, + DataTypes.BOOLEAN().notNull()), + resultSpec( + call( + "EQUALS", + call("LEAST", $("f0"), $("f1")), + call("LEAST", $("f0"), $("f1"))), + "LEAST(f0, f1) = LEAST(f0, f1)", + null, + DataTypes.BOOLEAN())) + .testSqlValidationError( + "LEAST(f5, f6)", + "SQL validation failed. Invalid function call:\n" + + "LEAST(STRING NOT NULL, TIMESTAMP(3) NOT NULL)")); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonAggregationFunctionsITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonAggregationFunctionsITCase.java index 9209a496505bf..b0ffd6cc5b4a2 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonAggregationFunctionsITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonAggregationFunctionsITCase.java @@ -27,7 +27,11 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import static org.apache.flink.runtime.state.StateBackendLoader.HASHMAP_STATE_BACKEND_NAME; +import static org.apache.flink.runtime.state.StateBackendLoader.ROCKSDB_STATE_BACKEND_NAME; import static org.apache.flink.table.api.DataTypes.INT; import static org.apache.flink.table.api.DataTypes.ROW; import static org.apache.flink.table.api.DataTypes.STRING; @@ -43,128 +47,154 @@ public class JsonAggregationFunctionsITCase extends BuiltInAggregateFunctionTest @Parameterized.Parameters(name = "{index}: {0}") public static List testData() throws Exception { - return Arrays.asList( - // JSON_OBJECTAGG - TestSpec.forFunction(BuiltInFunctionDefinitions.JSON_OBJECTAGG_NULL_ON_NULL) - .withDescription("Basic Aggregation") - .withSource( - ROW(STRING(), INT()), - Arrays.asList( - Row.ofKind(INSERT, "A", 1), - Row.ofKind(INSERT, "B", null), - Row.ofKind(INSERT, "C", 3))) - .testResult( - source -> "SELECT JSON_OBJECTAGG(f0 VALUE f1) FROM " + source, - source -> - source.select( - jsonObjectAgg(JsonOnNull.NULL, $("f0"), $("f1"))), - ROW(VARCHAR(2000).notNull()), - ROW(STRING().notNull()), - Collections.singletonList(Row.of("{\"A\":1,\"B\":null,\"C\":3}"))), - TestSpec.forFunction(BuiltInFunctionDefinitions.JSON_OBJECTAGG_ABSENT_ON_NULL) - .withDescription("Omits NULLs") - .withSource( - ROW(STRING(), INT()), - Arrays.asList( - Row.ofKind(INSERT, "A", 1), - Row.ofKind(INSERT, "B", null), - Row.ofKind(INSERT, "C", 3))) - .testResult( - source -> - "SELECT JSON_OBJECTAGG(f0 VALUE f1 ABSENT ON NULL) FROM " - + source, - source -> - source.select( - jsonObjectAgg(JsonOnNull.ABSENT, $("f0"), $("f1"))), - ROW(VARCHAR(2000).notNull()), - ROW(STRING().notNull()), - Collections.singletonList(Row.of("{\"A\":1,\"C\":3}"))), - TestSpec.forFunction(BuiltInFunctionDefinitions.JSON_OBJECTAGG_NULL_ON_NULL) - .withDescription("Retractions") - .withSource( - ROW(STRING(), INT()), - Arrays.asList( - Row.ofKind(INSERT, "A", 1), - Row.ofKind(INSERT, "B", 2), - Row.ofKind(INSERT, "C", 3), - Row.ofKind(DELETE, "B", 2))) - .testResult( - source -> "SELECT JSON_OBJECTAGG(f0 VALUE f1) FROM " + source, - source -> - source.select( - jsonObjectAgg(JsonOnNull.NULL, $("f0"), $("f1"))), - ROW(VARCHAR(2000).notNull()), - ROW(STRING().notNull()), - Collections.singletonList(Row.of("{\"A\":1,\"C\":3}"))), - TestSpec.forFunction(BuiltInFunctionDefinitions.JSON_OBJECTAGG_NULL_ON_NULL) - .withDescription("Group Aggregation") - .withSource( - ROW(INT(), STRING(), INT()), - Arrays.asList( - Row.ofKind(INSERT, 1, "A", 0), - Row.ofKind(INSERT, 1, "B", 0), - Row.ofKind(INSERT, 2, "A", 0), - Row.ofKind(INSERT, 2, "C", 0))) - .testResult( - source -> - "SELECT f0, JSON_OBJECTAGG(f1 VALUE f2) FROM " - + source - + " GROUP BY f0", - source -> - source.groupBy($("f0")) - .select( - $("f0"), + return Stream.of( + // JSON_OBJECTAGG + TestSpec.forFunction(BuiltInFunctionDefinitions.JSON_OBJECTAGG_NULL_ON_NULL) + .withDescription("Basic Aggregation") + .withSource( + ROW(STRING(), INT()), + Arrays.asList( + Row.ofKind(INSERT, "A", 1), + Row.ofKind(INSERT, "B", null), + Row.ofKind(INSERT, "C", 3))) + .testResult( + source -> + "SELECT JSON_OBJECTAGG(f0 VALUE f1) FROM " + source, + source -> + source.select( jsonObjectAgg( - JsonOnNull.NULL, $("f1"), $("f2"))), - ROW(INT(), VARCHAR(2000).notNull()), - ROW(INT(), STRING().notNull()), - Arrays.asList( - Row.of(1, "{\"A\":0,\"B\":0}"), - Row.of(2, "{\"A\":0,\"C\":0}"))), + JsonOnNull.NULL, $("f0"), $("f1"))), + ROW(VARCHAR(2000).notNull()), + ROW(STRING().notNull()), + Collections.singletonList( + Row.of("{\"A\":1,\"B\":null,\"C\":3}"))), + TestSpec.forFunction( + BuiltInFunctionDefinitions.JSON_OBJECTAGG_ABSENT_ON_NULL) + .withDescription("Omits NULLs") + .withSource( + ROW(STRING(), INT()), + Arrays.asList( + Row.ofKind(INSERT, "A", 1), + Row.ofKind(INSERT, "B", null), + Row.ofKind(INSERT, "C", 3))) + .testResult( + source -> + "SELECT JSON_OBJECTAGG(f0 VALUE f1 ABSENT ON NULL) FROM " + + source, + source -> + source.select( + jsonObjectAgg( + JsonOnNull.ABSENT, + $("f0"), + $("f1"))), + ROW(VARCHAR(2000).notNull()), + ROW(STRING().notNull()), + Collections.singletonList(Row.of("{\"A\":1,\"C\":3}"))), + TestSpec.forFunction(BuiltInFunctionDefinitions.JSON_OBJECTAGG_NULL_ON_NULL) + .withDescription("Retractions") + .withSource( + ROW(STRING(), INT()), + Arrays.asList( + Row.ofKind(INSERT, "A", 1), + Row.ofKind(INSERT, "B", 2), + Row.ofKind(INSERT, "C", 3), + Row.ofKind(DELETE, "B", 2))) + .testResult( + source -> + "SELECT JSON_OBJECTAGG(f0 VALUE f1) FROM " + source, + source -> + source.select( + jsonObjectAgg( + JsonOnNull.NULL, $("f0"), $("f1"))), + ROW(VARCHAR(2000).notNull()), + ROW(STRING().notNull()), + Collections.singletonList(Row.of("{\"A\":1,\"C\":3}"))), + TestSpec.forFunction(BuiltInFunctionDefinitions.JSON_OBJECTAGG_NULL_ON_NULL) + .withDescription("Group Aggregation") + .withSource( + ROW(INT(), STRING(), INT()), + Arrays.asList( + Row.ofKind(INSERT, 1, "A", 0), + Row.ofKind(INSERT, 1, "B", 0), + Row.ofKind(INSERT, 2, "A", 0), + Row.ofKind(INSERT, 2, "C", 0))) + .testResult( + source -> + "SELECT f0, JSON_OBJECTAGG(f1 VALUE f2) FROM " + + source + + " GROUP BY f0", + source -> + source.groupBy($("f0")) + .select( + $("f0"), + jsonObjectAgg( + JsonOnNull.NULL, + $("f1"), + $("f2"))), + ROW(INT(), VARCHAR(2000).notNull()), + ROW(INT(), STRING().notNull()), + Arrays.asList( + Row.of(1, "{\"A\":0,\"B\":0}"), + Row.of(2, "{\"A\":0,\"C\":0}"))), - // JSON_ARRAYAGG - TestSpec.forFunction(BuiltInFunctionDefinitions.JSON_ARRAYAGG_ABSENT_ON_NULL) - .withDescription("Basic Aggregation") - .withSource( - ROW(STRING()), - Arrays.asList( - Row.ofKind(INSERT, "A"), - Row.ofKind(INSERT, (String) null), - Row.ofKind(INSERT, "C"))) - .testResult( - source -> "SELECT JSON_ARRAYAGG(f0) FROM " + source, - source -> source.select(jsonArrayAgg(JsonOnNull.ABSENT, $("f0"))), - ROW(VARCHAR(2000).notNull()), - ROW(STRING().notNull()), - Collections.singletonList(Row.of("[\"A\",\"C\"]"))), - TestSpec.forFunction(BuiltInFunctionDefinitions.JSON_ARRAYAGG_NULL_ON_NULL) - .withDescription("Keeps NULLs") - .withSource( - ROW(STRING()), - Arrays.asList( - Row.ofKind(INSERT, "A"), - Row.ofKind(INSERT, (String) null), - Row.ofKind(INSERT, "C"))) - .testResult( - source -> "SELECT JSON_ARRAYAGG(f0 NULL ON NULL) FROM " + source, - source -> source.select(jsonArrayAgg(JsonOnNull.NULL, $("f0"))), - ROW(VARCHAR(2000).notNull()), - ROW(STRING().notNull()), - Collections.singletonList(Row.of("[\"A\",null,\"C\"]"))), - TestSpec.forFunction(BuiltInFunctionDefinitions.JSON_ARRAYAGG_ABSENT_ON_NULL) - .withDescription("Retractions") - .withSource( - ROW(INT()), - Arrays.asList( - Row.ofKind(INSERT, 1), - Row.ofKind(INSERT, 2), - Row.ofKind(INSERT, 3), - Row.ofKind(DELETE, 2))) - .testResult( - source -> "SELECT JSON_ARRAYAGG(f0) FROM " + source, - source -> source.select(jsonArrayAgg(JsonOnNull.ABSENT, $("f0"))), - ROW(VARCHAR(2000).notNull()), - ROW(STRING().notNull()), - Collections.singletonList(Row.of("[1,3]")))); + // JSON_ARRAYAGG + TestSpec.forFunction( + BuiltInFunctionDefinitions.JSON_ARRAYAGG_ABSENT_ON_NULL) + .withDescription("Basic Aggregation") + .withSource( + ROW(STRING()), + Arrays.asList( + Row.ofKind(INSERT, "A"), + Row.ofKind(INSERT, (String) null), + Row.ofKind(INSERT, "C"))) + .testResult( + source -> "SELECT JSON_ARRAYAGG(f0) FROM " + source, + source -> + source.select( + jsonArrayAgg(JsonOnNull.ABSENT, $("f0"))), + ROW(VARCHAR(2000).notNull()), + ROW(STRING().notNull()), + Collections.singletonList(Row.of("[\"A\",\"C\"]"))), + TestSpec.forFunction(BuiltInFunctionDefinitions.JSON_ARRAYAGG_NULL_ON_NULL) + .withDescription("Keeps NULLs") + .withSource( + ROW(STRING()), + Arrays.asList( + Row.ofKind(INSERT, "A"), + Row.ofKind(INSERT, (String) null), + Row.ofKind(INSERT, "C"))) + .testResult( + source -> + "SELECT JSON_ARRAYAGG(f0 NULL ON NULL) FROM " + + source, + source -> + source.select( + jsonArrayAgg(JsonOnNull.NULL, $("f0"))), + ROW(VARCHAR(2000).notNull()), + ROW(STRING().notNull()), + Collections.singletonList(Row.of("[\"A\",null,\"C\"]"))), + TestSpec.forFunction( + BuiltInFunctionDefinitions.JSON_ARRAYAGG_ABSENT_ON_NULL) + .withDescription("Retractions") + .withSource( + ROW(INT()), + Arrays.asList( + Row.ofKind(INSERT, 1), + Row.ofKind(INSERT, 2), + Row.ofKind(INSERT, 3), + Row.ofKind(DELETE, 2))) + .testResult( + source -> "SELECT JSON_ARRAYAGG(f0) FROM " + source, + source -> + source.select( + jsonArrayAgg(JsonOnNull.ABSENT, $("f0"))), + ROW(VARCHAR(2000).notNull()), + ROW(STRING().notNull()), + Collections.singletonList(Row.of("[1,3]")))) + .flatMap( + spec -> + Stream.of(HASHMAP_STATE_BACKEND_NAME, ROCKSDB_STATE_BACKEND_NAME) + .map(backend -> spec.copy().withStateBackend(backend))) + .collect(Collectors.toList()); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonFunctionsITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonFunctionsITCase.java index e01c0f74df8f8..9fb64e7e58bcb 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonFunctionsITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonFunctionsITCase.java @@ -57,7 +57,6 @@ import static org.apache.flink.table.api.DataTypes.TIMESTAMP; import static org.apache.flink.table.api.DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE; import static org.apache.flink.table.api.DataTypes.VARBINARY; -import static org.apache.flink.table.api.DataTypes.VARCHAR; import static org.apache.flink.table.api.Expressions.$; import static org.apache.flink.table.api.Expressions.call; import static org.apache.flink.table.api.Expressions.jsonArray; @@ -183,7 +182,7 @@ private static TestSpec jsonValueSpec() throws Exception { "JSON_VALUE(CAST(NULL AS STRING), 'lax $')", null, STRING(), - VARCHAR(2000)), + STRING()), // RETURNING + Supported Data Types resultSpec( @@ -191,7 +190,7 @@ private static TestSpec jsonValueSpec() throws Exception { "JSON_VALUE(f0, '$.type')", "account", STRING(), - VARCHAR(2000)), + STRING()), resultSpec( $("f0").jsonValue("$.activated", BOOLEAN()), "JSON_VALUE(f0, '$.activated' RETURNING BOOLEAN)", @@ -220,7 +219,7 @@ private static TestSpec jsonValueSpec() throws Exception { "JSON_VALUE(f0, 'lax $.invalid' NULL ON EMPTY ERROR ON ERROR)", null, STRING(), - VARCHAR(2000)), + STRING()), resultSpec( $("f0").jsonValue( "lax $.invalid", @@ -243,7 +242,7 @@ private static TestSpec jsonValueSpec() throws Exception { "JSON_VALUE(f0, 'strict $.invalid' ERROR ON EMPTY NULL ON ERROR)", null, STRING(), - VARCHAR(2000)), + STRING()), resultSpec( $("f0").jsonValue( "strict $.invalid", diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/casting/CastRulesTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/casting/CastRulesTest.java index 126127975ce94..f676f88ff8f39 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/casting/CastRulesTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/casting/CastRulesTest.java @@ -23,6 +23,7 @@ import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableException; import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.data.DecimalData; import org.apache.flink.table.data.GenericArrayData; import org.apache.flink.table.data.GenericMapData; import org.apache.flink.table.data.GenericRowData; @@ -106,9 +107,11 @@ class CastRulesTest { private static final ZoneId CET = ZoneId.of("CET"); private static final CastRule.Context CET_CONTEXT = - CastRule.Context.create(false, CET, Thread.currentThread().getContextClassLoader()); + CastRule.Context.create( + false, false, CET, Thread.currentThread().getContextClassLoader()); private static final CastRule.Context CET_CONTEXT_LEGACY = - CastRule.Context.create(true, CET, Thread.currentThread().getContextClassLoader()); + CastRule.Context.create( + false, true, CET, Thread.currentThread().getContextClassLoader()); private static final byte DEFAULT_POSITIVE_TINY_INT = (byte) 5; private static final byte DEFAULT_NEGATIVE_TINY_INT = (byte) -5; @@ -650,21 +653,26 @@ Stream testCases() { .fromCase(BOOLEAN(), false, fromString("FALSE")) .fromCaseLegacy(BOOLEAN(), true, fromString("true")) .fromCaseLegacy(BOOLEAN(), false, fromString("false")) - .fromCase(BINARY(2), new byte[] {0, 1}, fromString("0001")) + .fromCase(BINARY(2), new byte[] {0, 1}, fromString("\u0000\u0001")) .fromCaseLegacy(BINARY(2), new byte[] {0, 1}, fromString("\u0000\u0001")) - .fromCase(VARBINARY(3), new byte[] {0, 1, 2}, fromString("000102")) + .fromCase( + VARBINARY(3), + new byte[] {0, 1, 2}, + fromString("\u0000\u0001\u0002")) .fromCaseLegacy( VARBINARY(3), new byte[] {0, 1, 2}, fromString("\u0000\u0001\u0002")) - .fromCase(VARBINARY(5), new byte[] {0, -1, -2}, fromString("00fffe")) .fromCaseLegacy(VARBINARY(5), new byte[] {102, 111, 111}, fromString("foo")) - .fromCase( - BYTES(), - new byte[] {-123, 43, -4, 125, 5}, - fromString("852bfc7d05")) + .fromCaseLegacy(VARBINARY(5), new byte[] {102, 111, 111}, fromString("foo")) .fromCaseLegacy( BYTES(), new byte[] {70, 108, 105, 110, 107}, fromString("Flink")) + .fromCaseLegacy( + BYTES(), new byte[] {70, 108, 105, 110, 107}, fromString("Flink")) + .fromCasePrinting( + BYTES(), + new byte[] {70, 108, 105, 110, 107}, + fromString("x'466c696e6b'")) .fromCase(BOOLEAN(), true, StringData.fromString("TRUE")) .fromCase(BOOLEAN(), false, StringData.fromString("FALSE")) .fromCase( @@ -837,11 +845,11 @@ Stream testCases() { .fromCaseLegacy(BOOLEAN(), true, fromString("true")) .fromCase(BOOLEAN(), false, fromString("FALSE ")) .fromCaseLegacy(BOOLEAN(), false, fromString("false")) - .fromCase(BINARY(1), new byte[] {-12}, fromString("f4 ")) + .fromCase(BINARY(1), new byte[] {102}, fromString("f ")) .fromCaseLegacy(BINARY(1), new byte[] {102}, fromString("f")) - .fromCase(VARBINARY(1), new byte[] {23}, fromString("17 ")) + .fromCase(VARBINARY(1), new byte[] {33}, fromString("\u0021 ")) .fromCaseLegacy(VARBINARY(1), new byte[] {33}, fromString("\u0021")) - .fromCase(BYTES(), new byte[] {32}, fromString("20 ")) + .fromCase(BYTES(), new byte[] {32}, fromString(" ")) .fromCaseLegacy(BYTES(), new byte[] {32}, fromString(" ")) .fromCase(TINYINT(), (byte) -125, fromString("-125 ")) .fromCaseLegacy(TINYINT(), (byte) -125, fromString("-125")) @@ -862,20 +870,20 @@ Stream testCases() { CastTestSpecBuilder.testCastTo(CHAR(12)) .fromCase( BINARY(4), - new byte[] {-12, 32, 46, -72}, - fromString("f4202eb8 ")) + new byte[] {1, 11, 111, 2}, + fromString("\u0001\u000B\u006F\u0002 ")) .fromCaseLegacy( BINARY(4), new byte[] {1, 11, 111, 2}, fromString("\u0001\u000B\u006F\u0002")) - .fromCase(VARBINARY(4), new byte[] {1, 11, 22}, fromString("010b16 ")) + .fromCase( + VARBINARY(4), + new byte[] {1, 11, 22}, + fromString("\u0001\u000B\u0016 ")) .fromCaseLegacy( VARBINARY(4), new byte[] {1, 11, 22}, fromString("\u0001\u000B\u0016")) - .fromCase(BYTES(), new byte[] {1, 11}, fromString("010b ")) - .fromCaseLegacy( - BYTES(), new byte[] {1, 11, 111}, fromString("\u0001\u000B\u006F")) .fromCase( ARRAY(INT()), new GenericArrayData(new int[] {-1, 2, 3}), @@ -962,17 +970,28 @@ Stream testCases() { .fromCaseLegacy(BOOLEAN(), true, fromString("true")) .fromCase(BOOLEAN(), false, fromString("FAL")) .fromCaseLegacy(BOOLEAN(), false, fromString("false")) - .fromCase(BINARY(5), new byte[] {0, 1, 2, 3, 4}, fromString("000")) + .fromCase(BINARY(2), new byte[] {0, 1}, fromString("\u0000\u0001")) + .fromCaseLegacy(BINARY(1), new byte[] {0, 1}, fromString("\u0000\u0001")) + .fromCase( + BINARY(5), + new byte[] {0, 1, 2, 3, 4}, + fromString("\u0000\u0001\u0002")) .fromCaseLegacy( BINARY(5), new byte[] {0, 1, 2, 3, 4}, fromString("\u0000\u0001\u0002\u0003\u0004")) - .fromCase(VARBINARY(5), new byte[] {0, 1, 2, 3, 4}, fromString("000")) + .fromCase( + VARBINARY(5), + new byte[] {0, 1, 2, 3, 4}, + fromString("\u0000\u0001\u0002")) .fromCaseLegacy( VARBINARY(5), new byte[] {0, 1, 2, 3, 4}, fromString("\u0000\u0001\u0002\u0003\u0004")) - .fromCase(BYTES(), new byte[] {0, 1, 2, 3, 4}, fromString("000")) + .fromCase( + BYTES(), + new byte[] {0, 1, 2, 3, 4}, + fromString("\u0000\u0001\u0002")) .fromCaseLegacy( BYTES(), new byte[] {0, 1, 2, 3, 4}, @@ -1126,30 +1145,27 @@ Stream testCases() { .fromCase(DOUBLE(), 0.0d, false) .fromCase(DOUBLE(), -0.12345678d, true), CastTestSpecBuilder.testCastTo(BINARY(4)) - .fromCase(CHAR(4), fromString("66"), new byte[] {102, 0, 0, 0}) + .fromCase(CHAR(4), fromString("66"), new byte[] {54, 54, 0, 0}) + .fromCaseLegacy(CHAR(4), fromString("66"), new byte[] {54, 54}) + .fromCase(CHAR(3), fromString("foo"), new byte[] {102, 111, 111, 0}) .fromCaseLegacy(CHAR(3), fromString("foo"), new byte[] {102, 111, 111}) - .fromCase(CHAR(10), fromString("66A2"), new byte[] {102, -94, 0, 0}) + .fromCase(CHAR(10), fromString("66A2"), new byte[] {54, 54, 65, 50}) + .fromCaseLegacy(CHAR(10), fromString("66A2"), new byte[] {54, 54, 65, 50}) + .fromCase(CHAR(1), fromString("f"), new byte[] {102, 0, 0, 0}) .fromCaseLegacy(CHAR(1), fromString("f"), new byte[] {102}) - .fromCase(CHAR(16), fromString("12f4aBc7"), new byte[] {18, -12, -85, -57}) - .fromCaseLegacy(CHAR(3), fromString("f"), new byte[] {102}) - .fromCase(VARCHAR(8), fromString("bACd"), new byte[] {-70, -51, 0, 0}) + .fromCase(CHAR(16), fromString("12f4aBc7"), new byte[] {49, 50, 102, 52}) + .fromCase(CHAR(3), fromString("A f "), new byte[] {65, 32, 102, 32}) + .fromCase(VARCHAR(8), fromString("bAC"), new byte[] {98, 65, 67, 0}) + .fromCase(VARCHAR(5), fromString("Flink"), new byte[] {70, 108, 105, 110}) .fromCaseLegacy( VARCHAR(5), fromString("Flink"), new byte[] {70, 108, 105, 110, 107}) - .fromCase( - STRING(), - fromString("12f4ABc71232"), - new byte[] {18, -12, -85, -57}) + .fromCase(STRING(), fromString("Apache"), new byte[] {65, 112, 97, 99}) .fromCaseLegacy( STRING(), fromString("Apache"), new byte[] {65, 112, 97, 99, 104, 101}) - .fromCase(STRING(), fromString("12F4ab"), new byte[] {18, -12, -85, 0}) - .fromCaseLegacy(STRING(), fromString("bar"), new byte[] {98, 97, 114}) - .fail(STRING(), fromString("123"), TableException.class) - .fail(STRING(), fromString("12P9"), TableException.class) - .fail(STRING(), fromString("12 A9"), TableException.class) .fromCase(BINARY(2), new byte[] {1, 2}, new byte[] {1, 2, 0, 0}) .fromCaseLegacy(BINARY(2), new byte[] {1, 2}, new byte[] {1, 2}) .fromCase(VARBINARY(3), new byte[] {1, 2, 3}, new byte[] {1, 2, 3, 0}) @@ -1157,24 +1173,27 @@ Stream testCases() { .fromCase(BYTES(), new byte[] {1, 2, 3}, new byte[] {1, 2, 3, 0}) .fromCaseLegacy(BYTES(), new byte[] {1, 2, 3}, new byte[] {1, 2, 3}), CastTestSpecBuilder.testCastTo(VARBINARY(4)) - .fromCase(CHAR(4), fromString("c9"), new byte[] {-55}) + .fromCase(CHAR(4), fromString("c9"), new byte[] {99, 57}) + .fromCaseLegacy(CHAR(4), fromString("c9"), new byte[] {99, 57}) + .fromCase(CHAR(3), fromString("foo"), new byte[] {102, 111, 111}) .fromCaseLegacy(CHAR(3), fromString("foo"), new byte[] {102, 111, 111}) - .fromCase(VARCHAR(8), fromString("7de2"), new byte[] {125, -30}) + .fromCase(VARCHAR(8), fromString("7de2"), new byte[] {55, 100, 101, 50}) + .fromCaseLegacy( + VARCHAR(8), fromString("7de2"), new byte[] {55, 100, 101, 50}) + .fromCase(VARCHAR(5), fromString("Flink"), new byte[] {70, 108, 105, 110}) .fromCaseLegacy( VARCHAR(5), fromString("Flink"), new byte[] {70, 108, 105, 110, 107}) - .fromCase( + .fromCase(STRING(), fromString("12F4a bC7"), new byte[] {49, 50, 70, 52}) + .fromCaseLegacy( STRING(), - fromString("12F4abC71232"), - new byte[] {18, -12, -85, -57}) + fromString("12F4a bC7"), + new byte[] {49, 50, 70, 52, 97, 32, 98, 67, 55}) .fromCaseLegacy( STRING(), fromString("Apache"), new byte[] {65, 112, 97, 99, 104, 101}) - .fail(STRING(), fromString("123"), TableException.class) - .fail(STRING(), fromString("12P9"), TableException.class) - .fail(STRING(), fromString("12 A9"), TableException.class) // We assume that the input length is respected, therefore, no trimming is // applied .fromCase(BINARY(2), new byte[] {1, 2, 3, 4, 5}, new byte[] {1, 2, 3, 4, 5}) @@ -1189,24 +1208,37 @@ Stream testCases() { new byte[] {1, 2, 3, 4, 5}, new byte[] {1, 2, 3, 4, 5}), CastTestSpecBuilder.testCastTo(BYTES()) - .fromCase(CHAR(4), fromString("9C"), new byte[] {-100}) + .fromCase(CHAR(4), fromString("9C"), new byte[] {57, 67}) + .fromCaseLegacy(CHAR(4), fromString("9C"), new byte[] {57, 67}) + .fromCase(CHAR(3), fromString("foo"), new byte[] {102, 111, 111}) .fromCaseLegacy(CHAR(3), fromString("foo"), new byte[] {102, 111, 111}) - .fromCase(VARCHAR(8), fromString("3ee3"), new byte[] {62, -29}) + .fromCase(VARCHAR(8), fromString("3ee3"), new byte[] {51, 101, 101, 51}) + .fromCaseLegacy( + VARCHAR(8), fromString("3ee3"), new byte[] {51, 101, 101, 51}) + .fromCase( + VARCHAR(5), + fromString("Flink"), + new byte[] {70, 108, 105, 110, 107}) .fromCaseLegacy( VARCHAR(5), fromString("Flink"), new byte[] {70, 108, 105, 110, 107}) .fromCase( STRING(), - fromString("AAbbCcDdff"), - new byte[] {-86, -69, -52, -35, -1}) + fromString("AAbb Cc Dd"), + new byte[] {65, 65, 98, 98, 32, 67, 99, 32, 68, 100}) .fromCaseLegacy( + STRING(), + fromString("AAbb Cc Dd"), + new byte[] {65, 65, 98, 98, 32, 67, 99, 32, 68, 100}) + .fromCase( STRING(), fromString("Apache"), new byte[] {65, 112, 97, 99, 104, 101}) - .fail(STRING(), fromString("123"), TableException.class) - .fail(STRING(), fromString("12P9"), TableException.class) - .fail(STRING(), fromString("12 A9"), TableException.class), + .fromCaseLegacy( + STRING(), + fromString("Apache"), + new byte[] {65, 112, 97, 99, 104, 101}), CastTestSpecBuilder.testCastTo(DECIMAL(5, 3)) .fail(CHAR(3), fromString("foo"), TableException.class) .fail(VARCHAR(5), fromString("Flink"), TableException.class) @@ -1258,8 +1290,8 @@ Stream testCases() { CastTestSpecBuilder.testCastTo(ARRAY(BIGINT().nullable())) .fromCase( ARRAY(INT().nullable()), - new GenericArrayData(new Object[] {1, null, 2}), - new GenericArrayData(new Object[] {1L, null, 2L})), + new GenericArrayData(new Integer[] {1, null, 2}), + new GenericArrayData(new Long[] {1L, null, 2L})), CastTestSpecBuilder.testCastTo(ARRAY(BIGINT().notNull())) .fromCase( ARRAY(INT().notNull()), @@ -1273,7 +1305,42 @@ Stream testCases() { new GenericArrayData(new Integer[] {1, 2, null}), new GenericArrayData(new Integer[] {3}) }), - NullPointerException.class), + NullPointerException.class) + .fromCase( + ARRAY(ARRAY(INT().nullable())), + new GenericArrayData( + new GenericArrayData[] { + new GenericArrayData(new Integer[] {1, 2}), + new GenericArrayData(new Integer[] {3}) + }), + new GenericArrayData( + new GenericArrayData[] { + new GenericArrayData(new Long[] {1L, 2L}), + new GenericArrayData(new Long[] {3L}) + })), + CastTestSpecBuilder.testCastTo(ARRAY(ARRAY(DECIMAL(10, 2).notNull()))) + .fromCase( + ARRAY(ARRAY(INT().notNull())), + new GenericArrayData( + new GenericArrayData[] { + new GenericArrayData(new Integer[] {1, 2}), + new GenericArrayData(new Integer[] {3}) + }), + new GenericArrayData( + new GenericArrayData[] { + new GenericArrayData( + new Object[] { + DecimalData.fromBigDecimal( + BigDecimal.ONE, 10, 2), + DecimalData.fromBigDecimal( + new BigDecimal(2), 10, 2) + }), + new GenericArrayData( + new Object[] { + DecimalData.fromBigDecimal( + new BigDecimal(3), 10, 2) + }) + })), CastTestSpecBuilder.testCastTo(MAP(DOUBLE().notNull(), DOUBLE().notNull())) .fromCase( MAP(INT().nullable(), INT().nullable()), @@ -1512,6 +1579,20 @@ private CastTestSpecBuilder fromCase(DataType srcDataType, Object src, Object ta return fromCase( srcDataType, CastRule.Context.create( + false, + false, + DateTimeUtils.UTC_ZONE.toZoneId(), + Thread.currentThread().getContextClassLoader()), + src, + target); + } + + private CastTestSpecBuilder fromCasePrinting( + DataType srcDataType, Object src, Object target) { + return fromCase( + srcDataType, + CastRule.Context.create( + true, false, DateTimeUtils.UTC_ZONE.toZoneId(), Thread.currentThread().getContextClassLoader()), @@ -1524,6 +1605,7 @@ private CastTestSpecBuilder fromCaseLegacy( return fromCase( srcDataType, CastRule.Context.create( + false, true, DateTimeUtils.UTC_ZONE.toZoneId(), Thread.currentThread().getContextClassLoader()), @@ -1553,6 +1635,7 @@ private CastTestSpecBuilder fail( return fail( dataType, CastRule.Context.create( + false, false, DateTimeUtils.UTC_ZONE.toZoneId(), Thread.currentThread().getContextClassLoader()), diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/ForwardHashExchangeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/ForwardHashExchangeTest.java index fd68753bb42e1..b5edb3632f969 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/ForwardHashExchangeTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/ForwardHashExchangeTest.java @@ -36,6 +36,12 @@ public void before() { util = batchTestUtil(TableConfig.getDefault()); util.getStreamEnv().getConfig().setDynamicGraph(true); + util.tableEnv() + .getConfig() + .getConfiguration() + .set( + OptimizerConfigOptions.TABLE_OPTIMIZER_SOURCE_AGGREGATE_PUSHDOWN_ENABLED, + false); util.tableEnv() .executeSql( "CREATE TABLE T (\n" @@ -71,6 +77,20 @@ public void before() { + ")"); } + @Test + public void testRankWithHashShuffle() { + util.verifyExecPlan( + "SELECT * FROM (SELECT a, b, RANK() OVER(PARTITION BY a ORDER BY b) rk FROM T) WHERE rk <= 10"); + } + + @Test + public void testSortAggregateWithHashShuffle() { + util.tableEnv() + .getConfig() + .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg"); + util.verifyExecPlan(" SELECT a, SUM(b) AS b FROM T GROUP BY a"); + } + @Test public void testOverAggOnHashAggWithHashShuffle() { util.tableEnv() @@ -130,12 +150,45 @@ public void testHashAggOnHashJoinWithHashShuffle() { } @Test - public void testSortAggOnSortMergeJoinWithHashShuffle() { + public void testOnePhaseSortAggOnSortMergeJoinWithHashShuffle() { + util.tableEnv() + .getConfig() + .set( + ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, + "HashJoin,NestedLoopJoin,HashAgg"); + util.tableEnv() + .getConfig() + .set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "ONE_PHASE"); + util.verifyExecPlan( + "WITH r AS (SELECT * FROM T1, T2 WHERE a1 = a2 AND c1 LIKE 'He%')\n" + + "SELECT sum(b1) FROM r group by a1"); + } + + @Test + public void testTwoPhaseSortAggOnSortMergeJoinWithHashShuffle() { + util.tableEnv() + .getConfig() + .set( + ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, + "HashJoin,NestedLoopJoin,HashAgg"); + util.tableEnv() + .getConfig() + .set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE"); + util.verifyExecPlan( + "WITH r AS (SELECT * FROM T1, T2 WHERE a1 = a2 AND c1 LIKE 'He%')\n" + + "SELECT sum(b1) FROM r group by a1"); + } + + @Test + public void testAutoPhaseSortAggOnSortMergeJoinWithHashShuffle() { util.tableEnv() .getConfig() .set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin,HashAgg"); + util.tableEnv() + .getConfig() + .set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "AUTO"); util.verifyExecPlan( "WITH r AS (SELECT * FROM T1, T2 WHERE a1 = a2 AND c1 LIKE 'He%')\n" + "SELECT sum(b1) FROM r group by a1"); @@ -196,10 +249,13 @@ public void testRankOnHashAggWithGlobalShuffle() { } @Test - public void testRankOnSortAggWithHashShuffle() { + public void testRankOnOnePhaseSortAggWithHashShuffle() { util.tableEnv() .getConfig() - .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg"); + .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg"); + util.tableEnv() + .getConfig() + .set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "ONE_PHASE"); util.verifyExecPlan( "SELECT * FROM (\n" + " SELECT a, b, RANK() OVER(PARTITION BY a ORDER BY b) rk FROM (\n" @@ -209,10 +265,45 @@ public void testRankOnSortAggWithHashShuffle() { } @Test - public void testRankOnSortAggWithGlobalShuffle() { + public void testRankOnTwoPhaseSortAggWithHashShuffle() { util.tableEnv() .getConfig() - .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg"); + .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg"); + util.tableEnv() + .getConfig() + .set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE"); + util.verifyExecPlan( + "SELECT * FROM (\n" + + " SELECT a, b, RANK() OVER(PARTITION BY a ORDER BY b) rk FROM (\n" + + " SELECT a, SUM(b) AS b FROM T GROUP BY a\n" + + " )\n" + + " ) WHERE rk <= 10"); + } + + @Test + public void testRankOnOnePhaseSortAggWithGlobalShuffle() { + util.tableEnv() + .getConfig() + .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg"); + util.tableEnv() + .getConfig() + .set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "ONE_PHASE"); + util.verifyExecPlan( + "SELECT * FROM (\n" + + " SELECT b, RANK() OVER(ORDER BY b) rk FROM (\n" + + " SELECT SUM(b) AS b FROM T\n" + + " )\n" + + " ) WHERE rk <= 10"); + } + + @Test + public void testRankOnTwoPhaseSortAggWithGlobalShuffle() { + util.tableEnv() + .getConfig() + .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg"); + util.tableEnv() + .getConfig() + .set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE"); util.verifyExecPlan( "SELECT * FROM (\n" + " SELECT b, RANK() OVER(ORDER BY b) rk FROM (\n" diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/TransformationsTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/TransformationsTest.java index 31ac8b52d3771..b0667b14ff937 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/TransformationsTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/TransformationsTest.java @@ -110,7 +110,7 @@ public void testLegacyBatchValues() { @Test public void testLegacyUid() { final TableEnvironment env = - TableEnvironment.create(EnvironmentSettings.inStreamingMode().toConfiguration()); + TableEnvironment.create(EnvironmentSettings.inStreamingMode().getConfiguration()); env.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_LEGACY_TRANSFORMATION_UIDS, true); env.createTemporaryTable( diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityConflictResolverTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityConflictResolverTest.java index 73f62285ed68a..4bb01a0756bb9 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityConflictResolverTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityConflictResolverTest.java @@ -112,6 +112,7 @@ public void testDeadlockCausedByExchange() { BatchExecExchange exchange = new BatchExecExchange( + new Configuration(), InputProperty.builder() .requiredDistribution(InputProperty.ANY_DISTRIBUTION) .build(), diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSinkSpecSerdeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSinkSpecSerdeTest.java index 57a712f2ab367..d6857b515dfb2 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSinkSpecSerdeTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSinkSpecSerdeTest.java @@ -44,6 +44,7 @@ import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.utils.CatalogManagerMocks; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.parallel.Execution; @@ -57,8 +58,6 @@ import java.util.Map; import java.util.stream.Stream; -import static org.apache.flink.table.api.EnvironmentSettings.DEFAULT_BUILTIN_CATALOG; -import static org.apache.flink.table.api.EnvironmentSettings.DEFAULT_BUILTIN_DATABASE; import static org.apache.flink.table.api.config.TableConfigOptions.PLAN_COMPILE_CATALOG_OBJECTS; import static org.apache.flink.table.api.config.TableConfigOptions.PLAN_RESTORE_CATALOG_OBJECTS; import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; @@ -99,8 +98,8 @@ static Stream testDynamicTableSinkSpecSerde() { new DynamicTableSinkSpec( ContextResolvedTable.temporary( ObjectIdentifier.of( - DEFAULT_BUILTIN_CATALOG, - DEFAULT_BUILTIN_DATABASE, + CatalogManagerMocks.DEFAULT_CATALOG, + CatalogManagerMocks.DEFAULT_DATABASE, "MyTable"), new ResolvedCatalogTable(catalogTable1, resolvedSchema1)), null); @@ -129,8 +128,8 @@ static Stream testDynamicTableSinkSpecSerde() { new DynamicTableSinkSpec( ContextResolvedTable.temporary( ObjectIdentifier.of( - DEFAULT_BUILTIN_CATALOG, - DEFAULT_BUILTIN_DATABASE, + CatalogManagerMocks.DEFAULT_CATALOG, + CatalogManagerMocks.DEFAULT_DATABASE, "MyTable"), new ResolvedCatalogTable(catalogTable2, resolvedSchema2)), Arrays.asList( @@ -165,8 +164,8 @@ static Stream testDynamicTableSinkSpecSerde() { new DynamicTableSinkSpec( ContextResolvedTable.temporary( ObjectIdentifier.of( - DEFAULT_BUILTIN_CATALOG, - DEFAULT_BUILTIN_DATABASE, + CatalogManagerMocks.DEFAULT_CATALOG, + CatalogManagerMocks.DEFAULT_DATABASE, "MyTable"), new ResolvedCatalogTable(catalogTable3, resolvedSchema3)), Collections.singletonList( @@ -214,7 +213,10 @@ void testDynamicTableSinkSpecSerde(DynamicTableSinkSpec spec) throws IOException void testDynamicTableSinkSpecSerdeWithEnrichmentOptions() throws Exception { // Test model ObjectIdentifier identifier = - ObjectIdentifier.of(DEFAULT_BUILTIN_CATALOG, DEFAULT_BUILTIN_DATABASE, "my_table"); + ObjectIdentifier.of( + CatalogManagerMocks.DEFAULT_CATALOG, + CatalogManagerMocks.DEFAULT_DATABASE, + "my_table"); String formatPrefix = FactoryUtil.getFormatPrefix(FORMAT, TestFormatFactory.IDENTIFIER); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSourceSpecSerdeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSourceSpecSerdeTest.java index a15acb7716cd2..3764f6b6b1512 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSourceSpecSerdeTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSourceSpecSerdeTest.java @@ -23,6 +23,7 @@ import org.apache.flink.formats.testcsv.TestCsvFormatFactory; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.Schema; +import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.api.config.TableConfigOptions.CatalogPlanCompilation; import org.apache.flink.table.api.config.TableConfigOptions.CatalogPlanRestore; import org.apache.flink.table.catalog.CatalogManager; @@ -72,8 +73,6 @@ import java.util.Map; import java.util.stream.Stream; -import static org.apache.flink.table.api.EnvironmentSettings.DEFAULT_BUILTIN_CATALOG; -import static org.apache.flink.table.api.EnvironmentSettings.DEFAULT_BUILTIN_DATABASE; import static org.apache.flink.table.api.config.TableConfigOptions.PLAN_COMPILE_CATALOG_OBJECTS; import static org.apache.flink.table.api.config.TableConfigOptions.PLAN_RESTORE_CATALOG_OBJECTS; import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; @@ -114,8 +113,8 @@ public static Stream testDynamicTableSinkSpecSerde() { new DynamicTableSourceSpec( ContextResolvedTable.temporary( ObjectIdentifier.of( - DEFAULT_BUILTIN_CATALOG, - DEFAULT_BUILTIN_DATABASE, + TableConfigOptions.TABLE_CATALOG_NAME.defaultValue(), + TableConfigOptions.TABLE_DATABASE_NAME.defaultValue(), "MyTable"), new ResolvedCatalogTable(catalogTable1, resolvedSchema1)), null); @@ -154,8 +153,8 @@ public static Stream testDynamicTableSinkSpecSerde() { new DynamicTableSourceSpec( ContextResolvedTable.temporary( ObjectIdentifier.of( - DEFAULT_BUILTIN_CATALOG, - DEFAULT_BUILTIN_DATABASE, + TableConfigOptions.TABLE_CATALOG_NAME.defaultValue(), + TableConfigOptions.TABLE_DATABASE_NAME.defaultValue(), "MyTable"), new ResolvedCatalogTable(catalogTable2, resolvedSchema2)), Arrays.asList( @@ -274,7 +273,10 @@ void testDynamicTableSourceSpecSerde(DynamicTableSourceSpec spec) throws IOExcep void testDynamicTableSourceSpecSerdeWithEnrichmentOptions() throws Exception { // Test model ObjectIdentifier identifier = - ObjectIdentifier.of(DEFAULT_BUILTIN_CATALOG, DEFAULT_BUILTIN_DATABASE, "my_table"); + ObjectIdentifier.of( + TableConfigOptions.TABLE_CATALOG_NAME.defaultValue(), + TableConfigOptions.TABLE_DATABASE_NAME.defaultValue(), + "my_table"); String formatPrefix = FactoryUtil.getFormatPrefix(FORMAT, TestFormatFactory.IDENTIFIER); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LogicalTypeJsonSerdeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LogicalTypeJsonSerdeTest.java index 6e29c40f1651b..d27dbb5abe359 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LogicalTypeJsonSerdeTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LogicalTypeJsonSerdeTest.java @@ -19,7 +19,6 @@ package org.apache.flink.table.planner.plan.nodes.exec.serde; import org.apache.flink.api.common.typeutils.base.LocalDateTimeSerializer; -import org.apache.flink.configuration.Configuration; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.ValidationException; @@ -112,18 +111,17 @@ public void testLogicalTypeSerde(LogicalType logicalType) throws IOException { public void testIdentifierSerde() throws IOException { final DataTypeFactoryMock dataTypeFactoryMock = new DataTypeFactoryMock(); final TableConfig tableConfig = TableConfig.getDefault(); - final Configuration config = tableConfig.getConfiguration(); final CatalogManager catalogManager = preparedCatalogManager().dataTypeFactory(dataTypeFactoryMock).build(); final SerdeContext serdeContext = configuredSerdeContext(catalogManager, tableConfig); // minimal plan content - config.set(TableConfigOptions.PLAN_COMPILE_CATALOG_OBJECTS, IDENTIFIER); + tableConfig.set(TableConfigOptions.PLAN_COMPILE_CATALOG_OBJECTS, IDENTIFIER); final String minimalJson = toJson(serdeContext, STRUCTURED_TYPE); assertThat(minimalJson).isEqualTo("\"`default_catalog`.`default_database`.`MyType`\""); // catalog lookup with miss - config.set( + tableConfig.set( TableConfigOptions.PLAN_RESTORE_CATALOG_OBJECTS, TableConfigOptions.CatalogPlanRestore.IDENTIFIER); dataTypeFactoryMock.logicalType = Optional.empty(); @@ -131,7 +129,7 @@ public void testIdentifierSerde() throws IOException { .satisfies(anyCauseMatches(ValidationException.class, "No type found.")); // catalog lookup - config.set( + tableConfig.set( TableConfigOptions.PLAN_RESTORE_CATALOG_OBJECTS, TableConfigOptions.CatalogPlanRestore.IDENTIFIER); dataTypeFactoryMock.logicalType = Optional.of(STRUCTURED_TYPE); @@ -139,7 +137,7 @@ public void testIdentifierSerde() throws IOException { .isEqualTo(STRUCTURED_TYPE); // maximum plan content - config.set(TableConfigOptions.PLAN_COMPILE_CATALOG_OBJECTS, ALL); + tableConfig.set(TableConfigOptions.PLAN_COMPILE_CATALOG_OBJECTS, ALL); final String maximumJson = toJson(serdeContext, STRUCTURED_TYPE); final ObjectMapper mapper = new ObjectMapper(); final JsonNode maximumJsonNode = mapper.readTree(maximumJson); @@ -149,7 +147,7 @@ public void testIdentifierSerde() throws IOException { .isEqualTo("My original type."); // catalog lookup with miss - config.set( + tableConfig.set( TableConfigOptions.PLAN_RESTORE_CATALOG_OBJECTS, TableConfigOptions.CatalogPlanRestore.IDENTIFIER); dataTypeFactoryMock.logicalType = Optional.empty(); @@ -157,7 +155,7 @@ public void testIdentifierSerde() throws IOException { .satisfies(anyCauseMatches(ValidationException.class, "No type found.")); // catalog lookup - config.set( + tableConfig.set( TableConfigOptions.PLAN_RESTORE_CATALOG_OBJECTS, TableConfigOptions.CatalogPlanRestore.IDENTIFIER); dataTypeFactoryMock.logicalType = Optional.of(UPDATED_STRUCTURED_TYPE); @@ -165,7 +163,7 @@ public void testIdentifierSerde() throws IOException { .isEqualTo(UPDATED_STRUCTURED_TYPE); // no lookup - config.set( + tableConfig.set( TableConfigOptions.PLAN_RESTORE_CATALOG_OBJECTS, TableConfigOptions.CatalogPlanRestore.ALL); dataTypeFactoryMock.logicalType = Optional.of(UPDATED_STRUCTURED_TYPE); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LookupKeySerdeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LookupKeySerdeTest.java index 2c3f372eb7503..a1f48c9084087 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LookupKeySerdeTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LookupKeySerdeTest.java @@ -50,7 +50,7 @@ public void testLookupKey() throws IOException { CatalogManager catalogManager = CatalogManager.newBuilder() .classLoader(Thread.currentThread().getContextClassLoader()) - .config(tableConfig.getConfiguration()) + .config(tableConfig) .defaultCatalog("default_catalog", new GenericInMemoryCatalog("default_db")) .build(); FlinkContext flinkContext = diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.java index d98ac9414fb22..8f3a4196d7143 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.java @@ -118,7 +118,8 @@ public void setup() { + " id int,\n" + " deepNested row, nested2 row>,\n" + " metadata_1 int metadata,\n" - + " metadata_2 string metadata\n" + + " metadata_2 string metadata,\n" + + " metadata_3 as cast(metadata_1 as bigint)\n" + ") WITH (" + " 'connector' = 'values'," + " 'nested-projection-supported' = 'true'," @@ -207,6 +208,13 @@ public void testComplicatedNestedProject() { util().verifyRelPlan(sqlQuery); } + @Test + public void testProjectWithDuplicateMetadataKey() { + String sqlQuery = "SELECT id, metadata_3, metadata_1 FROM MetadataTable"; + + util().verifyRelPlan(sqlQuery); + } + @Test public void testNestProjectWithMetadata() { String sqlQuery = @@ -349,7 +357,7 @@ public void testProjectionIncludingOnlyMetadata() { equalTo(Collections.emptyList())); assertThat( DataType.getFieldNames(appliedMetadataDataType.get()), - equalTo(Collections.singletonList("m2"))); + equalTo(Collections.singletonList("metadata"))); } @Test @@ -375,7 +383,7 @@ public void testProjectionWithMetadataAndPhysicalFields() { equalTo(Collections.singletonList("f1"))); assertThat( DataType.getFieldNames(appliedMetadataDataType.get()), - equalTo(Arrays.asList("f1", "m2"))); + equalTo(Arrays.asList("f1", "metadata"))); } // --------------------------------------------------------------------------------------------- diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.java index 51aee49f5badf..b7671622fa71c 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.java @@ -124,8 +124,7 @@ public void testDisablePushDownLocalAgg() { // disable push down local agg util.getTableEnv() .getConfig() - .getConfiguration() - .setBoolean( + .set( OptimizerConfigOptions.TABLE_OPTIMIZER_SOURCE_AGGREGATE_PUSHDOWN_ENABLED, false); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/LegacyTableFactoryTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/LegacyTableFactoryTest.java new file mode 100644 index 0000000000000..2984e74086dd1 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/LegacyTableFactoryTest.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.stream.sql; + +import org.apache.flink.connector.file.table.LegacyTableFactory; +import org.apache.flink.table.planner.utils.JavaStreamTableTestUtil; +import org.apache.flink.table.planner.utils.TableTestBase; + +import org.junit.Test; + +/** Tests for usages of {@link LegacyTableFactory}. */ +public class LegacyTableFactoryTest extends TableTestBase { + + private final JavaStreamTableTestUtil util; + + public LegacyTableFactoryTest() { + util = javaStreamTestUtil(); + util.tableEnv().executeSql("CREATE TABLE T (a INT) WITH ('type'='legacy')"); + } + + @Test + public void testSelect() { + util.verifyExecPlan("SELECT * FROM T"); + } + + @Test + public void testInsert() { + util.verifyExecPlanInsert("INSERT INTO T VALUES (1)"); + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtilTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtilTest.java index 0c59dbe02acc0..9301f79e72516 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtilTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtilTest.java @@ -20,6 +20,9 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; @@ -42,7 +45,9 @@ import java.util.List; import java.util.Set; import java.util.stream.Collectors; +import java.util.stream.Stream; +import static org.apache.flink.configuration.ConfigOptions.key; import static org.apache.flink.table.planner.plan.utils.ExecNodeMetadataUtil.UNSUPPORTED_JSON_SERDE_CLASSES; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -104,6 +109,26 @@ public void testMultipleAnnotations() { new Condition<>( m -> m.minPlanVersion() == FlinkVersion.v1_15, "minStateVersion")); + Configuration config = new Configuration(); + config.set(OPTION_1, 1); + config.set(OPTION_2, 2); + config.set(OPTION_3, 3); + config.set(OPTION_4, 4); + config.set(OPTION_5, 5); + config.set(OPTION_6, 6); + + ReadableConfig persistedConfig = + ExecNodeMetadataUtil.newPersistedConfig( + DummyNode.class, + config, + Stream.of(OPTION_1, OPTION_2, OPTION_3, OPTION_4, OPTION_5, OPTION_6)); + assertThat(persistedConfig.get(OPTION_1)).isEqualTo(1); + assertThat(persistedConfig.get(OPTION_2)).isEqualTo(OPTION_2.defaultValue()); + assertThat(persistedConfig.get(OPTION_3)).isEqualTo(3); + assertThat(persistedConfig.get(OPTION_4)).isEqualTo(4); + assertThat(persistedConfig.get(OPTION_5)).isEqualTo(5); + assertThat(persistedConfig.get(OPTION_6)).isEqualTo(OPTION_6.defaultValue()); + // Using multiple individual ExecNodeMetadata annotations ExecNodeMetadataUtil.addTestNode(DummyNodeMultipleAnnotations.class); assertThat(ExecNodeMetadataUtil.retrieveExecNode("dummy-node-multiple-annotations", 1)) @@ -120,6 +145,73 @@ public void testMultipleAnnotations() { .has( new Condition<>( m -> m.minPlanVersion() == FlinkVersion.v1_15, "minStateVersion")); + + assertThatThrownBy( + () -> + ExecNodeContext.newPersistedConfig( + DummyNodeMultipleAnnotations.class, config)) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "ExecNode: org.apache.flink.table.planner.plan.utils." + + "ExecNodeMetadataUtilTest.DummyNodeMultipleAnnotations, " + + "consumedOption: option111 not listed in [TableConfigOptions, " + + "ExecutionConfigOptions]."); + } + + @Test + public void testDuplicateConsumedOptions() { + ExecNodeMetadataUtil.addTestNode(DummyNodeDuplicateConsumedOptions.class); + assertThatThrownBy( + () -> + ExecNodeMetadataUtil.newPersistedConfig( + DummyNodeDuplicateConsumedOptions.class, + new Configuration(), + Stream.of(OPTION_1, OPTION_2, OPTION_3))) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "ExecNode: org.apache.flink.table.planner.plan.utils." + + "ExecNodeMetadataUtilTest." + + "DummyNodeDuplicateConsumedOptions, consumedOption: " + + "option2 is listed multiple times in consumedOptions, " + + "potentially also with fallback/deprecated key."); + } + + @Test + public void testDuplicateDeprecatedKeysConsumedOptions() { + ExecNodeMetadataUtil.addTestNode(DummyNodeDuplicateDeprecatedKeysConsumedOptions.class); + assertThatThrownBy( + () -> + ExecNodeMetadataUtil.newPersistedConfig( + DummyNodeDuplicateDeprecatedKeysConsumedOptions.class, + new Configuration(), + Stream.of(OPTION_1, OPTION_2, OPTION_3))) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "ExecNode: org.apache.flink.table.planner.plan.utils." + + "ExecNodeMetadataUtilTest." + + "DummyNodeDuplicateDeprecatedKeysConsumedOptions, " + + "consumedOption: option3-deprecated is listed multiple times in " + + "consumedOptions, potentially also with fallback/deprecated " + + "key."); + } + + @Test + public void testDuplicateFallbackKeysConsumedOptions() { + ExecNodeMetadataUtil.addTestNode(DummyNodeDuplicateFallbackKeysConsumedOptions.class); + assertThatThrownBy( + () -> + ExecNodeMetadataUtil.newPersistedConfig( + DummyNodeDuplicateFallbackKeysConsumedOptions.class, + new Configuration(), + Stream.of(OPTION_1, OPTION_2, OPTION_4))) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "ExecNode: org.apache.flink.table.planner.plan.utils." + + "ExecNodeMetadataUtilTest." + + "DummyNodeDuplicateFallbackKeysConsumedOptions, " + + "consumedOption: option4-fallback is listed multiple times in " + + "consumedOptions, potentially also with fallback/deprecated " + + "key."); } @Test @@ -128,15 +220,16 @@ public void testNewContext() { .isInstanceOf(IllegalStateException.class) .hasMessage( "ExecNode: org.apache.flink.table.planner.plan.utils." - + "ExecNodeMetadataUtilTest.DummyNodeNoAnnotation is not listed in the " - + "unsupported classes since it is not annotated with: ExecNodeMetadata."); + + "ExecNodeMetadataUtilTest.DummyNodeNoAnnotation is not " + + "listed in the unsupported classes since it is not annotated " + + "with: ExecNodeMetadata."); assertThatThrownBy(() -> ExecNodeContext.newContext(DummyNode.class)) .isInstanceOf(IllegalStateException.class) .hasMessage( "ExecNode: org.apache.flink.table.planner.plan.utils." - + "ExecNodeMetadataUtilTest.DummyNode is not listed in the supported " - + "classes and yet is annotated with: ExecNodeMetadata."); + + "ExecNodeMetadataUtilTest.DummyNode is not listed in the " + + "supported classes and yet is annotated with: ExecNodeMetadata."); } @Test @@ -178,103 +271,89 @@ public void testStreamExecNodeJsonSerdeCoverage() { @ExecNodeMetadata( name = "dummy-node", version = 1, + consumedOptions = {"option1", "option3-deprecated", "option5-deprecated"}, minPlanVersion = FlinkVersion.v1_13, minStateVersion = FlinkVersion.v1_13), @ExecNodeMetadata( name = "dummy-node", version = 2, + consumedOptions = {"option2", "option3-deprecated", "option5", "option6-fallback"}, minPlanVersion = FlinkVersion.v1_14, minStateVersion = FlinkVersion.v1_14), @ExecNodeMetadata( name = "dummy-node", version = 3, + consumedOptions = {"option1", "option3", "option4-fallback", "option5-deprecated"}, minPlanVersion = FlinkVersion.v1_15, minStateVersion = FlinkVersion.v1_15) }) - private static class DummyNode extends ExecNodeBase { + private static class DummyNode extends AbstractDummyNode { @JsonCreator protected DummyNode( ExecNodeContext context, + ReadableConfig persistedConfig, List properties, LogicalType outputType, String description) { - super(10, context, properties, outputType, description); - } - - @Override - protected Transformation translateToPlanInternal( - PlannerBase planner, ExecNodeConfig config) { - return null; + super(context, persistedConfig, properties, outputType, description); } } @ExecNodeMetadata( name = "dummy-node-multiple-annotations", version = 1, + consumedOptions = {"option1", "option2"}, minPlanVersion = FlinkVersion.v1_13, minStateVersion = FlinkVersion.v1_13) @ExecNodeMetadata( name = "dummy-node-multiple-annotations", version = 2, + consumedOptions = {"option11", "option22"}, minPlanVersion = FlinkVersion.v1_14, minStateVersion = FlinkVersion.v1_14) @ExecNodeMetadata( name = "dummy-node-multiple-annotations", version = 3, + consumedOptions = {"option111", "option222"}, minPlanVersion = FlinkVersion.v1_15, minStateVersion = FlinkVersion.v1_15) - private static class DummyNodeMultipleAnnotations extends ExecNodeBase { + private static class DummyNodeMultipleAnnotations extends AbstractDummyNode { @JsonCreator protected DummyNodeMultipleAnnotations( ExecNodeContext context, + ReadableConfig persistedConfig, List properties, LogicalType outputType, String description) { - super(10, context, properties, outputType, description); - } - - @Override - protected Transformation translateToPlanInternal( - PlannerBase planner, ExecNodeConfig config) { - return null; + super(context, persistedConfig, properties, outputType, description); } } - private static class DummyNodeNoJsonCreator extends ExecNodeBase { + private static class DummyNodeNoJsonCreator extends AbstractDummyNode { protected DummyNodeNoJsonCreator( ExecNodeContext context, + ReadableConfig persistedConfig, List properties, LogicalType outputType, String description) { - super(10, context, properties, outputType, description); - } - - @Override - protected Transformation translateToPlanInternal( - PlannerBase planner, ExecNodeConfig config) { - return null; + super(context, persistedConfig, properties, outputType, description); } } - private static class DummyNodeNoAnnotation extends ExecNodeBase + private static class DummyNodeNoAnnotation extends AbstractDummyNode implements StreamExecNode { @JsonCreator protected DummyNodeNoAnnotation( ExecNodeContext context, + ReadableConfig persistedConfig, List properties, LogicalType outputType, String description) { - super(10, context, properties, outputType, description); - } - - @Override - protected Transformation translateToPlanInternal( - PlannerBase planner, ExecNodeConfig config) { - return null; + super(context, persistedConfig, properties, outputType, description); } } @@ -295,15 +374,85 @@ protected Transformation translateToPlanInternal( version = 3, minPlanVersion = FlinkVersion.v1_15, minStateVersion = FlinkVersion.v1_15) - private static class DummyNodeBothAnnotations extends ExecNodeBase { + private static class DummyNodeBothAnnotations extends AbstractDummyNode { @JsonCreator protected DummyNodeBothAnnotations( ExecNodeContext context, + ReadableConfig persistedConfig, List properties, LogicalType outputType, String description) { - super(10, context, properties, outputType, description); + super(context, persistedConfig, properties, outputType, description); + } + } + + @ExecNodeMetadata( + name = "dummy-node-duplicate-consumedOptions", + version = 3, + consumedOptions = {"option1", "option2", "option3", "option2"}, + minPlanVersion = FlinkVersion.v1_15, + minStateVersion = FlinkVersion.v1_15) + private static class DummyNodeDuplicateConsumedOptions extends AbstractDummyNode { + + @JsonCreator + protected DummyNodeDuplicateConsumedOptions( + ExecNodeContext context, + ReadableConfig persistedConfig, + List properties, + LogicalType outputType, + String description) { + super(context, persistedConfig, properties, outputType, description); + } + } + + @ExecNodeMetadata( + name = "dummy-node-duplicate-deprecated-keys-consumedOptions", + version = 3, + consumedOptions = {"option1", "option2", "option3", "option3-deprecated"}, + minPlanVersion = FlinkVersion.v1_15, + minStateVersion = FlinkVersion.v1_15) + private static class DummyNodeDuplicateDeprecatedKeysConsumedOptions extends AbstractDummyNode { + + @JsonCreator + protected DummyNodeDuplicateDeprecatedKeysConsumedOptions( + ExecNodeContext context, + ReadableConfig persistedConfig, + List properties, + LogicalType outputType, + String description) { + super(context, persistedConfig, properties, outputType, description); + } + } + + @ExecNodeMetadata( + name = "dummy-node-duplicate-fallback-keys-consumedOptions", + version = 3, + consumedOptions = {"option1", "option2", "option4", "option4-fallback"}, + minPlanVersion = FlinkVersion.v1_15, + minStateVersion = FlinkVersion.v1_15) + private static class DummyNodeDuplicateFallbackKeysConsumedOptions extends AbstractDummyNode { + + @JsonCreator + protected DummyNodeDuplicateFallbackKeysConsumedOptions( + ExecNodeContext context, + ReadableConfig persistedConfig, + List properties, + LogicalType outputType, + String description) { + super(context, persistedConfig, properties, outputType, description); + } + } + + private static class AbstractDummyNode extends ExecNodeBase { + + protected AbstractDummyNode( + ExecNodeContext context, + ReadableConfig persistedConfig, + List properties, + LogicalType outputType, + String description) { + super(10, context, persistedConfig, properties, outputType, description); } @Override @@ -312,4 +461,35 @@ protected Transformation translateToPlanInternal( return null; } } + + private static final ConfigOption OPTION_1 = + key("option1").intType().defaultValue(-1).withDescription("option1"); + private static final ConfigOption OPTION_2 = + key("option2").intType().defaultValue(-1).withDescription("option2"); + private static final ConfigOption OPTION_3 = + key("option3") + .intType() + .defaultValue(-1) + .withDeprecatedKeys("option3-deprecated") + .withDescription("option3"); + private static final ConfigOption OPTION_4 = + key("option4") + .intType() + .defaultValue(-1) + .withFallbackKeys("option4-fallback") + .withDescription("option4"); + private static final ConfigOption OPTION_5 = + key("option5") + .intType() + .defaultValue(-1) + .withFallbackKeys("option5-fallback") + .withDeprecatedKeys("option5-deprecated") + .withDescription("option5"); + private static final ConfigOption OPTION_6 = + key("option6") + .intType() + .defaultValue(-1) + .withDeprecatedKeys("option6-deprecated") + .withFallbackKeys("option6-fallback") + .withDescription("option6"); } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/CorrelateJsonPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/CorrelateJsonPlanITCase.java index 18ae4e98ebba1..a12a14f402a2b 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/CorrelateJsonPlanITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/CorrelateJsonPlanITCase.java @@ -99,7 +99,9 @@ public void testFilter() throws ExecutionException, InterruptedException, IOExce "STRING_SPLIT", new JavaUserDefinedTableFunctions.StringSplit()); createTestValuesSinkTable("MySink", "a STRING", "b STRING"); String query = - "insert into MySink SELECT a, v FROM MyTable, lateral table(STRING_SPLIT(a, ',')) as T(v) where cast(v as int) > 0"; + "insert into MySink " + + "SELECT a, v FROM MyTable, lateral table(STRING_SPLIT(a, ',')) as T(v) " + + "where try_cast(v as int) > 0"; compileSqlAndExecutePlan(query).await(); List expected = Arrays.asList("+I[1,1,hi, 1]", "+I[1,1,hi, 1]"); assertResult(expected, TestValuesTableFactory.getResults("MySink")); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java index d39d180f1b5c5..ed19aba020a7b 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java @@ -64,6 +64,7 @@ import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -925,16 +926,12 @@ public void testInvalidUseOfScalarFunction() { public void testInvalidUseOfSystemScalarFunction() { tEnv().executeSql("CREATE TABLE SinkTable(s STRING) WITH ('connector' = 'COLLECTION')"); - try { - tEnv().explainSql("INSERT INTO SinkTable " + "SELECT * FROM TABLE(MD5('3'))"); - fail(); - } catch (ValidationException e) { - assertThat( - e, - hasMessage( - containsString( - "Currently, only table functions can be used in a correlate operation."))); - } + assertThatThrownBy( + () -> + tEnv().explainSql( + "INSERT INTO SinkTable " + + "SELECT * FROM TABLE(MD5('3'))")) + .hasMessageContaining("Argument must be a table function: MD5"); } @Test @@ -946,16 +943,12 @@ public void testInvalidUseOfTableFunction() { tEnv().createTemporarySystemFunction("RowTableFunction", RowTableFunction.class); - try { - tEnv().explainSql("INSERT INTO SinkTable " + "SELECT RowTableFunction('test')"); - fail(); - } catch (ValidationException e) { - assertThat( - e, - hasMessage( - containsString( - "Currently, only scalar functions can be used in a projection or filter operation."))); - } + assertThatThrownBy( + () -> + tEnv().explainSql( + "INSERT INTO SinkTable " + + "SELECT RowTableFunction('test')")) + .hasMessageContaining("Cannot call table function here: 'RowTableFunction'"); } @Test diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/table/FunctionITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/table/FunctionITCase.java index 6891d62c68b74..06af8ef372c01 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/table/FunctionITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/table/FunctionITCase.java @@ -28,27 +28,23 @@ import org.apache.flink.table.planner.runtime.utils.StreamingTestBase; import org.apache.flink.types.Row; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; +import org.junit.jupiter.api.Test; import java.util.Arrays; import java.util.List; +import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; import static org.apache.flink.table.api.Expressions.$; import static org.apache.flink.table.api.Expressions.call; -import static org.hamcrest.CoreMatchers.containsString; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.hamcrest.CoreMatchers.equalTo; import static org.junit.Assert.assertThat; -import static org.junit.internal.matchers.ThrowableMessageMatcher.hasMessage; /** Tests for user defined functions in the Table API. */ public class FunctionITCase extends StreamingTestBase { - @Rule public ExpectedException thrown = ExpectedException.none(); - @Test - public void testScalarFunction() throws Exception { + void testScalarFunction() throws Exception { final List sourceData = Arrays.asList(Row.of(1, 1L, 1L), Row.of(2, 2L, 1L), Row.of(3, 3L, 1L)); @@ -61,7 +57,7 @@ public void testScalarFunction() throws Exception { tEnv().executeSql( "CREATE TABLE TestTable(a INT, b BIGINT, c BIGINT) WITH ('connector' = 'COLLECTION')"); - Table table = + final Table table = tEnv().from("TestTable") .select( $("a"), @@ -75,7 +71,7 @@ public void testScalarFunction() throws Exception { } @Test - public void testJoinWithTableFunction() throws Exception { + void testJoinWithTableFunction() throws Exception { final List sourceData = Arrays.asList( Row.of("1,2,3"), Row.of("2,3,4"), Row.of("3,4,5"), Row.of((String) null)); @@ -103,23 +99,22 @@ public void testJoinWithTableFunction() throws Exception { } @Test - public void testLateralJoinWithScalarFunction() throws Exception { - thrown.expect(ValidationException.class); - thrown.expect( - hasMessage( - containsString( - "Currently, only table functions can be used in a correlate operation."))); - + void testLateralJoinWithScalarFunction() throws Exception { TestCollectionTableFactory.reset(); tEnv().executeSql("CREATE TABLE SourceTable(s STRING) WITH ('connector' = 'COLLECTION')"); tEnv().executeSql( "CREATE TABLE SinkTable(s STRING, sa ARRAY) WITH ('connector' = 'COLLECTION')"); - tEnv().from("SourceTable") - .joinLateral(call(new RowScalarFunction(), $("s")).as("a", "b")) - .select($("a"), $("b")) - .executeInsert("SinkTable") - .await(); + assertThatThrownBy( + () -> { + tEnv().from("SourceTable") + .joinLateral( + call(new RowScalarFunction(), $("s")).as("a", "b")); + }) + .satisfies( + anyCauseMatches( + ValidationException.class, + "A lateral join only accepts an expression which defines a table function")); } // -------------------------------------------------------------------------------------------- diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedTableFunctions.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedTableFunctions.java index 95f6e51e14ef5..9d7d559713b86 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedTableFunctions.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedTableFunctions.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.tuple.Tuple12; +import org.apache.flink.table.annotation.DataTypeHint; import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.functions.TableFunction; @@ -34,7 +35,10 @@ public class JavaUserDefinedTableFunctions { /** Emit inputs as long. */ public static class JavaTableFunc0 extends TableFunction { - public void eval(Integer a, Long b, TimestampData c) { + public void eval( + @DataTypeHint("DATE") Integer a, + Long b, + @DataTypeHint("TIMESTAMP(0)") TimestampData c) { collect(a.longValue()); collect(b); collect(c.getMillisecond()); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/JsonPlanTestBase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/JsonPlanTestBase.java index f175fccba8982..7415ee80caa46 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/JsonPlanTestBase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/JsonPlanTestBase.java @@ -22,6 +22,7 @@ import org.apache.flink.streaming.api.transformations.UnionTransformation; import org.apache.flink.table.api.CompiledPlan; import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.PlanReference; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableResult; import org.apache.flink.table.api.internal.CompiledPlanUtils; @@ -73,7 +74,10 @@ public void after() { protected TableResult compileSqlAndExecutePlan(String sql) { CompiledPlan compiledPlan = tableEnv.compilePlanSql(sql); checkTransformationUids(compiledPlan); - return compiledPlan.execute(); + // try to execute the string json plan to validate to ser/de result + String jsonPlan = compiledPlan.asJsonString(); + CompiledPlan newCompiledPlan = tableEnv.loadPlan(PlanReference.fromJsonString(jsonPlan)); + return newCompiledPlan.execute(); } protected void checkTransformationUids(CompiledPlan compiledPlan) { diff --git a/flink-table/flink-table-planner/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink-table/flink-table-planner/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory index 9a8051b6a8917..b929b23a97705 100644 --- a/flink-table/flink-table-planner/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory +++ b/flink-table/flink-table-planner/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory @@ -35,3 +35,4 @@ org.apache.flink.table.planner.utils.TestDataTypeTableSourceWithTimeFactory org.apache.flink.table.planner.utils.TestStreamTableSourceFactory org.apache.flink.table.planner.utils.TestFileInputFormatTableSourceFactory org.apache.flink.table.planner.utils.TestTableSourceWithTimeFactory +org.apache.flink.connector.file.table.LegacyTableFactory diff --git a/flink-table/flink-table-planner/src/test/resources/jsonplan/testGetJsonPlan.out b/flink-table/flink-table-planner/src/test/resources/jsonplan/testGetJsonPlan.out index 1bc32adf98a61..281372b28f5aa 100644 --- a/flink-table/flink-table-planner/src/test/resources/jsonplan/testGetJsonPlan.out +++ b/flink-table/flink-table-planner/src/test/resources/jsonplan/testGetJsonPlan.out @@ -40,6 +40,12 @@ { "id": 2, "type": "stream-exec-sink_1", + "configuration":{ + "table.exec.sink.keyed-shuffle":"AUTO", + "table.exec.sink.not-null-enforcer":"ERROR", + "table.exec.sink.type-length-enforcer":"IGNORE", + "table.exec.sink.upsert-materialize":"AUTO" + }, "dynamicTableSink": { "table": { "identifier": "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/connector/file/table/FileSystemTableSourceTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/connector/file/table/FileSystemTableSourceTest.xml index 6a2c8f88d9b5f..41096c54733cf 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/connector/file/table/FileSystemTableSourceTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/connector/file/table/FileSystemTableSourceTest.xml @@ -50,8 +50,8 @@ LogicalSink(table=[default_catalog.default_database.MySink], fields=[a, b, filem diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/CalcTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/CalcTest.xml index 5cbb2330a72e7..77ae802044512 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/CalcTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/CalcTest.xml @@ -156,6 +156,24 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/ForwardHashExchangeTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/ForwardHashExchangeTest.xml index 960e1165e4863..898426eb39efd 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/ForwardHashExchangeTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/ForwardHashExchangeTest.xml @@ -16,6 +16,38 @@ See the License for the specific language governing permissions and limitations under the License. --> + + + + + + + + + + + + + + + + + + + + + + @@ -229,7 +293,8 @@ OverAggregate(orderBy=[b ASC], window#0=[RANK(*) AS w0$o0 RANG BETWEEN UNBOUNDED +- Exchange(distribution=[forward]) +- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS b]) +- Exchange(distribution=[single]) - +- TableSourceScan(table=[[default_catalog, default_database, T, project=[b], metadata=[], aggregates=[grouping=[], aggFunctions=[LongSumAggFunction(b)]]]], fields=[sum$0]) + +- LocalHashAggregate(select=[Partial_SUM(b) AS sum$0]) + +- TableSourceScan(table=[[default_catalog, default_database, T, project=[b], metadata=[]]], fields=[b]) ]]> @@ -260,7 +325,8 @@ Calc(select=[sum_b, (CASE((w0$o0 > 0), w0$o1, null:BIGINT) / w0$o0) AS avg_b, w1 +- Exchange(distribution=[keep_input_as_is[hash[c]]]) +- HashAggregate(isMerge=[true], groupBy=[c], select=[c, Final_SUM(sum$0) AS sum_b]) +- Exchange(distribution=[hash[c]]) - +- TableSourceScan(table=[[default_catalog, default_database, T, project=[c, b], metadata=[], aggregates=[grouping=[c], aggFunctions=[LongSumAggFunction(b)]]]], fields=[c, sum$0]) + +- LocalHashAggregate(groupBy=[c], select=[c, Partial_SUM(b) AS sum$0]) + +- TableSourceScan(table=[[default_catalog, default_database, T, project=[c, b], metadata=[]]], fields=[c, b]) ]]> @@ -284,7 +350,8 @@ OverAggregate(orderBy=[b ASC], window#0=[RANK(*) AS w0$o0 RANG BETWEEN UNBOUNDED +- Exchange(distribution=[forward]) +- SortAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS b]) +- Exchange(distribution=[single]) - +- TableSourceScan(table=[[default_catalog, default_database, T, project=[b], metadata=[], aggregates=[grouping=[], aggFunctions=[LongSumAggFunction(b)]]]], fields=[sum$0]) + +- LocalSortAggregate(select=[Partial_SUM(b) AS sum$0]) + +- TableSourceScan(table=[[default_catalog, default_database, T, project=[b], metadata=[]]], fields=[b]) ]]> @@ -315,7 +382,10 @@ Calc(select=[sum_b, (CASE((w0$o0 > 0), w0$o1, null:BIGINT) / w0$o0) AS avg_b, w1 +- Exchange(distribution=[forward]) +- Sort(orderBy=[c ASC]) +- Exchange(distribution=[hash[c]]) - +- TableSourceScan(table=[[default_catalog, default_database, T, project=[c, b], metadata=[], aggregates=[grouping=[c], aggFunctions=[LongSumAggFunction(b)]]]], fields=[c, sum$0]) + +- LocalSortAggregate(groupBy=[c], select=[c, Partial_SUM(b) AS sum$0]) + +- Exchange(distribution=[forward]) + +- Sort(orderBy=[c ASC]) + +- TableSourceScan(table=[[default_catalog, default_database, T, project=[c, b], metadata=[]]], fields=[c, b]) ]]> @@ -345,7 +415,8 @@ Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=10], partitionBy=[], order +- Exchange(distribution=[forward]) +- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS b]) +- Exchange(distribution=[single]) - +- TableSourceScan(table=[[default_catalog, default_database, T, project=[b], metadata=[], aggregates=[grouping=[], aggFunctions=[LongSumAggFunction(b)]]]], fields=[sum$0]) + +- LocalHashAggregate(select=[Partial_SUM(b) AS sum$0]) + +- TableSourceScan(table=[[default_catalog, default_database, T, project=[b], metadata=[]]], fields=[b]) ]]> @@ -375,11 +446,12 @@ Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=10], partitionBy=[a], orde +- Exchange(distribution=[keep_input_as_is[hash[a]]]) +- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS b]) +- Exchange(distribution=[hash[a]]) - +- TableSourceScan(table=[[default_catalog, default_database, T, project=[a, b], metadata=[], aggregates=[grouping=[a], aggFunctions=[LongSumAggFunction(b)]]]], fields=[a, sum$0]) + +- LocalHashAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0]) + +- TableSourceScan(table=[[default_catalog, default_database, T, project=[a, b], metadata=[]]], fields=[a, b]) ]]> - + - + - + - + + + + + + + + + + + + + + + + + + + + + + + @@ -509,7 +631,31 @@ SortAggregate(isMerge=[false], select=[SUM(b1) AS EXPR$0]) ]]> - + + + + + + + + + + + + @@ -528,16 +674,63 @@ LogicalProject(EXPR$0=[$1]) + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.xml index d34b6e84fc807..7b89148eab2f5 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.xml @@ -57,8 +57,8 @@ LogicalIntersect(all=[true]) vcol_right_cnt), vcol_right_cnt, vcol_left_cnt) AS $f0, c], where=[((vcol_left_cnt >= 1) AND (vcol_right_cnt >= 1))]) +- HashAggregate(isMerge=[true], groupBy=[c], select=[c, Final_COUNT(count$0) AS vcol_left_cnt, Final_COUNT(count$1) AS vcol_right_cnt]) +- Exchange(distribution=[hash[c]]) @@ -180,8 +180,8 @@ LogicalMinus(all=[true]) 0)]) +- HashAggregate(isMerge=[true], groupBy=[c], select=[c, Final_SUM(sum$0) AS sum_vcol_marker]) +- Exchange(distribution=[hash[c]]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.xml index d6347f7086dce..a2d90a00ed11a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.xml @@ -26,13 +26,13 @@ LogicalProject(c=[$0], d=[$1]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], rowType=[RecordType(VARCHAR(2147483647) d, INTEGER e)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], rowType=[RecordType(VARCHAR(2147483647) d, INTEGER e)]) ]]> ($1, 20)]) ++- Correlate(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], correlate=[table(*org.apache.flink.table.planner.utils.TableFunc0*(c))], select=[a,b,c,d,e], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) d, INTEGER e)], joinType=[INNER], condition=[>($1, 20)]) +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -42,12 +42,12 @@ Calc(select=[c, d]) @@ -62,13 +62,13 @@ LogicalProject(c=[$0], d=[$1]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], rowType=[RecordType(VARCHAR(2147483647) d, INTEGER e)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], rowType=[RecordType(VARCHAR(2147483647) d, INTEGER e)]) ]]> ($1, 20)]) ++- Correlate(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], correlate=[table(*org.apache.flink.table.planner.utils.TableFunc0*(c))], select=[a,b,c,d,e], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) d, INTEGER e)], joinType=[INNER], condition=[>($1, 20)]) +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -79,13 +79,13 @@ Calc(select=[c, d]) LogicalProject(c=[$2], s=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)]) ]]> @@ -96,13 +96,13 @@ Calc(select=[c, s]) LogicalProject(c=[$2], s=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2, _UTF-16LE'$')], rowType=[RecordType(VARCHAR(2147483647) s)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2, _UTF-16LE'$')], rowType=[RecordType(VARCHAR(2147483647) s)]) ]]> @@ -114,13 +114,13 @@ LogicalFilter(condition=[>($1, _UTF-16LE'')]) +- LogicalProject(c=[$2], s=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[left], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)]) ]]> '')]) -+- Correlate(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], correlate=[table(TableFunc1(c))], select=[a,b,c,s], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)], joinType=[LEFT]) ++- Correlate(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], correlate=[table(*org.apache.flink.table.planner.utils.TableFunc1*(c))], select=[a,b,c,s], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)], joinType=[LEFT]) +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -131,13 +131,13 @@ Calc(select=[c, s], where=[(s > '')]) LogicalProject(c=[$2], s=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[left], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/SetOperatorsTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/SetOperatorsTest.xml index 598ee0e9af542..d31bda912a895 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/SetOperatorsTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/SetOperatorsTest.xml @@ -33,8 +33,8 @@ Calc(select=[EXPR$0 AS a, b, EXPR$1 AS c]) +- HashAggregate(isMerge=[true], groupBy=[b], select=[b, Final_SUM(sum$0) AS EXPR$0, Final_COUNT(count$1) AS EXPR$1]) +- Exchange(distribution=[hash[b]]) +- LocalHashAggregate(groupBy=[b], select=[b, Partial_SUM(a) AS sum$0, Partial_COUNT(c) AS count$1]) - +- Calc(select=[f0 AS a, f1 AS b, f2 AS c]) - +- Correlate(invocation=[$REPLICATE_ROWS$1($0, $1, $2, $3)], correlate=[table($REPLICATE_ROWS$1(sum_vcol_marker,a,b,c))], select=[sum_vcol_marker,a,b,c,f0,f1,f2], rowType=[RecordType(BIGINT sum_vcol_marker, INTEGER a, BIGINT b, VARCHAR(2147483647) c, INTEGER f0, BIGINT f1, VARCHAR(2147483647) f2)], joinType=[INNER]) + +- Calc(select=[a0 AS a, b0 AS b, c0 AS c]) + +- Correlate(invocation=[$REPLICATE_ROWS$1($0, $1, $2, $3)], correlate=[table($REPLICATE_ROWS$1(sum_vcol_marker,a,b,c))], select=[sum_vcol_marker,a,b,c,a0,b0,c0], rowType=[RecordType(BIGINT sum_vcol_marker, INTEGER a, BIGINT b, VARCHAR(2147483647) c, INTEGER a0, BIGINT b0, VARCHAR(2147483647) c0)], joinType=[INNER]) +- Calc(select=[sum_vcol_marker, a, b, c], where=[(sum_vcol_marker > 0)]) +- HashAggregate(isMerge=[true], groupBy=[a, b, c], select=[a, b, c, Final_SUM(sum$0) AS sum_vcol_marker]) +- Exchange(distribution=[hash[a, b, c]]) @@ -123,8 +123,8 @@ LogicalProject(b=[$1], c=[$2]) 0)]) +- HashAggregate(isMerge=[true], groupBy=[b, c], select=[b, c, Final_SUM(sum$0) AS sum_vcol_marker]) +- Exchange(distribution=[hash[b, c]]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/stringexpr/CorrelateStringExpressionTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/stringexpr/CorrelateStringExpressionTest.xml index 09262da4ea9fa..e69868781a231 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/stringexpr/CorrelateStringExpressionTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/stringexpr/CorrelateStringExpressionTest.xml @@ -22,13 +22,13 @@ limitations under the License. LogicalProject(c=[$2], s=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)]) ]]> @@ -39,13 +39,13 @@ Calc(select=[c, s]) LogicalProject(c=[$2], s=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[left], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)]) ]]> @@ -56,13 +56,13 @@ Calc(select=[c, s]) LogicalProject(c=[$2], s=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2, _UTF-16LE'$')], rowType=[RecordType(VARCHAR(2147483647) s)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2, _UTF-16LE'$')], rowType=[RecordType(VARCHAR(2147483647) s)]) ]]> @@ -73,13 +73,13 @@ Calc(select=[c, s]) LogicalProject(c=[$2], name=[$3], len=[$4]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc2*($2)], rowType=[RecordType(VARCHAR(2147483647) name, INTEGER len)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc2*($2)], rowType=[RecordType(VARCHAR(2147483647) name, INTEGER len)]) ]]> @@ -90,13 +90,13 @@ Calc(select=[c, name, len]) LogicalProject(c=[$2], name=[$3], len=[$5], adult=[$4]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.HierarchyTableFunction*($2)], rowType=[RecordType(VARCHAR(2147483647) name, BOOLEAN adult, INTEGER len)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.HierarchyTableFunction*($2)], rowType=[RecordType(VARCHAR(2147483647) name, BOOLEAN adult, INTEGER len)]) ]]> @@ -104,16 +104,16 @@ Calc(select=[c, name, len, adult]) @@ -125,13 +125,13 @@ LogicalFilter(condition=[>($2, 2)]) +- LogicalProject(c=[$2], name=[$3], len=[$4]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc2*($2)], rowType=[RecordType(VARCHAR(2147483647) name, INTEGER len)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc2*($2)], rowType=[RecordType(VARCHAR(2147483647) name, INTEGER len)]) ]]> ($1, 2)]) ++- Correlate(invocation=[*org.apache.flink.table.planner.utils.TableFunc2*($2)], correlate=[table(*org.apache.flink.table.planner.utils.TableFunc2*(c))], select=[a,b,c,name,len], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) name, INTEGER len)], joinType=[INNER], condition=[>($1, 2)]) +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -142,13 +142,13 @@ Calc(select=[c, name, len]) LogicalProject(a=[$0], c=[$2], s=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*(SUBSTRING($2, 2))], rowType=[RecordType(VARCHAR(2147483647) s)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*(SUBSTRING($2, 2))], rowType=[RecordType(VARCHAR(2147483647) s)]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/common/PartialInsertTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/common/PartialInsertTest.xml index 067ed8e287aa1..1ede5abe8397a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/common/PartialInsertTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/common/PartialInsertTest.xml @@ -127,8 +127,8 @@ LogicalSink(table=[default_catalog.default_database.partitioned_sink], fields=[a (sum_vcol_marker, 0)]) +- GroupAggregate(groupBy=[a, c, d, e, f, g], select=[a, c, d, e, f, g, SUM_RETRACT(vcol_marker) AS sum_vcol_marker]) +- Exchange(distribution=[hash[a, c, d, e, f, g]]) @@ -165,8 +165,8 @@ LogicalSink(table=[default_catalog.default_database.partitioned_sink], fields=[a (sum_vcol_marker, 0)]) +- HashAggregate(isMerge=[true], groupBy=[a, c, d, e, f, g], select=[a, c, d, e, f, g, Final_SUM(sum$0) AS sum_vcol_marker]) +- Exchange(distribution=[hash[a, c, d, e, f, g]]) @@ -205,8 +205,8 @@ LogicalSink(table=[default_catalog.default_database.partitioned_sink], fields=[a (vcol_left_cnt, vcol_right_cnt), vcol_right_cnt, vcol_left_cnt) AS $f0, a, c, d, e, f, g], where=[AND(>=(vcol_left_cnt, 1), >=(vcol_right_cnt, 1))]) +- GroupAggregate(groupBy=[a, c, d, e, f, g], select=[a, c, d, e, f, g, COUNT_RETRACT(vcol_left_marker) AS vcol_left_cnt, COUNT_RETRACT(vcol_right_marker) AS vcol_right_cnt]) +- Exchange(distribution=[hash[a, c, d, e, f, g]]) @@ -243,8 +243,8 @@ LogicalSink(table=[default_catalog.default_database.partitioned_sink], fields=[a (vcol_left_cnt, vcol_right_cnt), vcol_right_cnt, vcol_left_cnt) AS $f0, a, c, d, e, f, g], where=[AND(>=(vcol_left_cnt, 1), >=(vcol_right_cnt, 1))]) +- HashAggregate(isMerge=[true], groupBy=[a, c, d, e, f, g], select=[a, c, d, e, f, g, Final_COUNT(count$0) AS vcol_left_cnt, Final_COUNT(count$1) AS vcol_right_cnt]) +- Exchange(distribution=[hash[a, c, d, e, f, g]]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testComplexCalc.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testComplexCalc.out index 9ac59d9ec43d8..4dcf71adccd78 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testComplexCalc.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testComplexCalc.out @@ -208,6 +208,12 @@ }, { "id" : 3, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleFilter.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleFilter.out index c104d9a4a3370..6028828ca81f6 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleFilter.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleFilter.out @@ -85,6 +85,12 @@ }, { "id" : 3, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleProject.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleProject.out index 2ae314d98f9dc..13806b710dce4 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleProject.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleProject.out @@ -46,6 +46,12 @@ }, { "id" : 2, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testChangelogSource.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testChangelogSource.out index 63449c095d367..1f4ca9d8dd78a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testChangelogSource.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testChangelogSource.out @@ -77,6 +77,10 @@ }, { "id" : 4, "type" : "stream-exec-changelog-normalize_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "uniqueKeys" : [ 0, 1 ], "generateUpdateBefore" : true, "inputProperties" : [ { @@ -91,6 +95,12 @@ }, { "id" : 5, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testUpsertSource.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testUpsertSource.out index b89ceb2ee9656..81bd0cc7972b7 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testUpsertSource.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testUpsertSource.out @@ -65,6 +65,10 @@ }, { "id" : 3, "type" : "stream-exec-changelog-normalize_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "uniqueKeys" : [ 0, 1 ], "generateUpdateBefore" : true, "inputProperties" : [ { @@ -79,6 +83,12 @@ }, { "id" : 4, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoin.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoin.out index ba4d03d672baa..89307808f00f7 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoin.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoin.out @@ -87,6 +87,12 @@ }, { "id" : 4, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoinOverrideParameters.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoinOverrideParameters.out index c8def35787512..9e69b436938ca 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoinOverrideParameters.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoinOverrideParameters.out @@ -91,6 +91,12 @@ }, { "id" : 4, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testJoinWithFilter.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testJoinWithFilter.out index c31983bf6ef73..033efb8ede049 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testJoinWithFilter.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testJoinWithFilter.out @@ -101,6 +101,12 @@ }, { "id" : 4, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testLeftOuterJoinWithLiteralTrue.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testLeftOuterJoinWithLiteralTrue.out index 8f151698837f8..7c3ebe89595f5 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testLeftOuterJoinWithLiteralTrue.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testLeftOuterJoinWithLiteralTrue.out @@ -87,6 +87,12 @@ }, { "id" : 4, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/DeduplicationJsonPlanTest_jsonplan/testDeduplication.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/DeduplicationJsonPlanTest_jsonplan/testDeduplication.out index 9829b7cdd0e2f..f2c4a58820d1e 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/DeduplicationJsonPlanTest_jsonplan/testDeduplication.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/DeduplicationJsonPlanTest_jsonplan/testDeduplication.out @@ -154,6 +154,12 @@ }, { "id" : 4, "type" : "stream-exec-deduplicate_1", + "configuration" : { + "table.exec.deduplicate.insert-update-after-sensitive-enabled" : "true", + "table.exec.deduplicate.mini-batch.compact-changes-enabled" : "false", + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "uniqueKeys" : [ 2 ], "isRowtime" : false, "keepLastRow" : false, @@ -223,6 +229,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`sink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ExpandJsonPlanTest_jsonplan/testExpand.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ExpandJsonPlanTest_jsonplan/testExpand.out index 04d059e3aac90..3982472c711a4 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ExpandJsonPlanTest_jsonplan/testExpand.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ExpandJsonPlanTest_jsonplan/testExpand.out @@ -232,6 +232,10 @@ }, { "id" : 6, "type" : "stream-exec-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "grouping" : [ 0, 3, 4 ], "aggCalls" : [ { "name" : null, @@ -281,6 +285,10 @@ }, { "id" : 8, "type" : "stream-exec-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : null, @@ -316,6 +324,12 @@ }, { "id" : 9, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=false].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=false].out index 5a691039e310f..52aad805789e8 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=false].out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=false].out @@ -94,6 +94,10 @@ }, { "id" : 4, "type" : "stream-exec-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "cnt_a1", @@ -240,6 +244,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=true].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=true].out index d56ae50aa590f..bfef96f23e12b 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=true].out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=true].out @@ -97,6 +97,10 @@ }, { "id" : 4, "type" : "stream-exec-local-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "cnt_a1", @@ -337,6 +341,10 @@ }, { "id" : 6, "type" : "stream-exec-global-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "cnt_a1", @@ -484,6 +492,12 @@ }, { "id" : 8, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=false].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=false].out index 54e1aa4fc9eee..4ca7313f0a283 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=false].out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=false].out @@ -99,6 +99,10 @@ }, { "id" : 4, "type" : "stream-exec-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "cnt_a", @@ -192,6 +196,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=true].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=true].out index 6d0adeb903fc2..b31d031ffd0c5 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=true].out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=true].out @@ -102,6 +102,10 @@ }, { "id" : 4, "type" : "stream-exec-local-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "cnt_a", @@ -159,6 +163,10 @@ }, { "id" : 6, "type" : "stream-exec-global-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "cnt_a", @@ -253,6 +261,12 @@ }, { "id" : 8, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=false].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=false].out index c8d7b01d1a088..ab5ecc94b196d 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=false].out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=false].out @@ -104,6 +104,10 @@ }, { "id" : 4, "type" : "stream-exec-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "grouping" : [ ], "aggCalls" : [ { "name" : "avg_a", @@ -216,6 +220,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=true].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=true].out index c53e46390e100..8e9e1e6e283af 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=true].out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=true].out @@ -108,6 +108,10 @@ }, { "id" : 4, "type" : "stream-exec-local-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ ], "aggCalls" : [ { "name" : "avg_a", @@ -173,6 +177,10 @@ }, { "id" : 6, "type" : "stream-exec-global-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ ], "aggCalls" : [ { "name" : "avg_a", @@ -286,6 +294,12 @@ }, { "id" : 8, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=false].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=false].out index cc12b898c21aa..1ce190c4f7c0f 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=false].out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=false].out @@ -88,6 +88,10 @@ }, { "id" : 4, "type" : "stream-exec-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "a1", @@ -182,6 +186,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=true].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=true].out index 41687d09b8d7a..201e9547f11b6 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=true].out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=true].out @@ -104,6 +104,10 @@ }, { "id" : 5, "type" : "stream-exec-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "a1", @@ -198,6 +202,12 @@ }, { "id" : 7, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out index aef0a949c30dd..ab0ae9d10881a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out @@ -200,6 +200,11 @@ }, { "id" : 5, "type" : "stream-exec-group-window-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1", + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$1", @@ -259,6 +264,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeSessionWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeSessionWindow.out index a0a44cb80049c..467b00f4df578 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeSessionWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeSessionWindow.out @@ -200,6 +200,11 @@ }, { "id" : 5, "type" : "stream-exec-group-window-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1", + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$1", @@ -257,6 +262,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out index f41b8d274cb50..d30a69a2d5fd1 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out @@ -200,6 +200,11 @@ }, { "id" : 5, "type" : "stream-exec-group-window-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1", + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$3", @@ -410,6 +415,12 @@ }, { "id" : 7, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out index 1c9fbb79b63a2..68ecda4bd6e07 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out @@ -287,6 +287,11 @@ }, { "id" : 6, "type" : "stream-exec-group-window-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1", + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$1", @@ -338,6 +343,12 @@ }, { "id" : 7, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindow.out index b44c9c4daf429..e07c84acb3004 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindow.out @@ -287,6 +287,11 @@ }, { "id" : 6, "type" : "stream-exec-group-window-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1", + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$1", @@ -336,6 +341,12 @@ }, { "id" : 7, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out index d9c1ed0437798..bc5fd3fa7c035 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out @@ -266,6 +266,11 @@ }, { "id" : 6, "type" : "stream-exec-group-window-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1", + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$2", @@ -407,6 +412,12 @@ }, { "id" : 8, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregate.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregate.out index 76cc64fec94c3..a2bcf705ab28a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregate.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregate.out @@ -102,6 +102,10 @@ }, { "id" : 4, "type" : "stream-exec-local-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ 0, 2 ], "aggCalls" : [ { "name" : null, @@ -209,6 +213,10 @@ }, { "id" : 6, "type" : "stream-exec-incremental-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "partialAggGrouping" : [ 0, 1 ], "finalAggGrouping" : [ 0 ], "partialOriginalAggCalls" : [ { @@ -250,6 +258,10 @@ }, { "id" : 8, "type" : "stream-exec-global-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : null, @@ -277,6 +289,12 @@ }, { "id" : 9, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregateWithSumCountDistinctAndRetraction.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregateWithSumCountDistinctAndRetraction.out index abb17ca57f619..c4a0f219d8ecb 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregateWithSumCountDistinctAndRetraction.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregateWithSumCountDistinctAndRetraction.out @@ -62,6 +62,10 @@ }, { "id" : 3, "type" : "stream-exec-local-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "b", @@ -110,6 +114,10 @@ }, { "id" : 5, "type" : "stream-exec-global-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "b", @@ -187,6 +195,10 @@ }, { "id" : 7, "type" : "stream-exec-local-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ 0, 2 ], "aggCalls" : [ { "name" : null, @@ -315,6 +327,10 @@ }, { "id" : 9, "type" : "stream-exec-incremental-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "partialAggGrouping" : [ 0, 1 ], "finalAggGrouping" : [ 0 ], "partialOriginalAggCalls" : [ { @@ -375,6 +391,10 @@ }, { "id" : 11, "type" : "stream-exec-global-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : null, @@ -421,6 +441,12 @@ }, { "id" : 12, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testProcessingTimeInnerJoinWithOnClause.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testProcessingTimeInnerJoinWithOnClause.out index 05bdf0d776dab..4d61dede144bd 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testProcessingTimeInnerJoinWithOnClause.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testProcessingTimeInnerJoinWithOnClause.out @@ -656,6 +656,12 @@ }, { "id" : 13, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testRowTimeInnerJoinWithOnClause.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testRowTimeInnerJoinWithOnClause.out index d20fb04851b23..94927bb53e7b2 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testRowTimeInnerJoinWithOnClause.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testRowTimeInnerJoinWithOnClause.out @@ -480,6 +480,12 @@ }, { "id" : 11, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoin.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoin.out index fae9d801614d9..0355fb03137ad 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoin.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoin.out @@ -136,6 +136,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithEqualPk.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithEqualPk.out index a75de5cd4e0be..5d80fb2baac57 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithEqualPk.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithEqualPk.out @@ -56,6 +56,10 @@ }, { "id" : 3, "type" : "stream-exec-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "grouping" : [ 0 ], "aggCalls" : [ ], "aggCallNeedRetractions" : [ ], @@ -139,6 +143,10 @@ }, { "id" : 7, "type" : "stream-exec-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "grouping" : [ 0 ], "aggCalls" : [ ], "aggCallNeedRetractions" : [ ], @@ -196,6 +204,12 @@ }, { "id" : 10, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithPk.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithPk.out index 3bd92217d6de9..149408c5e1c60 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithPk.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithPk.out @@ -56,6 +56,10 @@ }, { "id" : 3, "type" : "stream-exec-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "a2", @@ -170,6 +174,10 @@ }, { "id" : 8, "type" : "stream-exec-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "b2", @@ -288,6 +296,12 @@ }, { "id" : 13, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testLeftJoinNonEqui.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testLeftJoinNonEqui.out index 289eb09c4edc1..27df7d82b1ccd 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testLeftJoinNonEqui.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testLeftJoinNonEqui.out @@ -172,6 +172,12 @@ }, { "id" : 7, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitJsonPlanTest_jsonplan/testLimit.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitJsonPlanTest_jsonplan/testLimit.out index aa1c8c2c7be9d..361b644af179b 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitJsonPlanTest_jsonplan/testLimit.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitJsonPlanTest_jsonplan/testLimit.out @@ -61,6 +61,9 @@ }, { "id" : 3, "type" : "stream-exec-limit_1", + "configuration" : { + "table.exec.rank.topn-cache-size" : "10000" + }, "rankRange" : { "type" : "Constant", "start" : 1, @@ -112,6 +115,12 @@ }, { "id" : 5, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTable.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTable.out index 4f4afabbd6f39..cba8ae4894a33 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTable.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTable.out @@ -378,6 +378,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithProjectionPushDown.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithProjectionPushDown.out index 17946aba99ea8..c3012c79471f5 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithProjectionPushDown.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithProjectionPushDown.out @@ -368,6 +368,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testMatch.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testMatch.out index b921155e86d97..62fc80c1cbc50 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testMatch.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testMatch.out @@ -130,11 +130,11 @@ "pattern" : { "kind" : "CALL", "syntax" : "BINARY", - "internalName" : "$$1", + "sqlKind" : "PATTERN_CONCAT", "operands" : [ { "kind" : "CALL", "syntax" : "BINARY", - "internalName" : "$$1", + "sqlKind" : "PATTERN_CONCAT", "operands" : [ { "kind" : "LITERAL", "value" : "A\"", @@ -302,6 +302,12 @@ }, { "id" : 5, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedNonPartitionedRangeOver.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedNonPartitionedRangeOver.out index 2541c41215928..4a91e46f98eaf 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedNonPartitionedRangeOver.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedNonPartitionedRangeOver.out @@ -362,6 +362,12 @@ }, { "id" : 8, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRangeOver.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRangeOver.out index 461837d4bb880..15f5426ec9924 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRangeOver.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRangeOver.out @@ -422,6 +422,12 @@ }, { "id" : 8, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime.out index fba1d4819773f..5d722f46d04df 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime.out @@ -359,6 +359,12 @@ }, { "id" : 7, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeUnboundedPartitionedRangeOver.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeUnboundedPartitionedRangeOver.out index 6f7c396ea6489..c4a29465ec854 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeUnboundedPartitionedRangeOver.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeUnboundedPartitionedRangeOver.out @@ -393,6 +393,12 @@ }, { "id" : 8, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctPartitionedRowOver.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctPartitionedRowOver.out index 6c8d61cfd199a..4eaf0b63f03ee 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctPartitionedRowOver.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctPartitionedRowOver.out @@ -403,6 +403,12 @@ }, { "id" : 8, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctWithNonDistinctPartitionedRowOver.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctWithNonDistinctPartitionedRowOver.out index 3e8b12df51628..b7477b9ae92f4 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctWithNonDistinctPartitionedRowOver.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctWithNonDistinctPartitionedRowOver.out @@ -493,6 +493,12 @@ }, { "id" : 8, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testRowTimeBoundedPartitionedRowsOver.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testRowTimeBoundedPartitionedRowsOver.out index cd7d26f2fe809..9eb9209956eb4 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testRowTimeBoundedPartitionedRowsOver.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testRowTimeBoundedPartitionedRowsOver.out @@ -237,6 +237,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/RankJsonPlanTest_jsonplan/testRank.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/RankJsonPlanTest_jsonplan/testRank.out index 5a3a6e6b26b53..3c0fde4021af0 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/RankJsonPlanTest_jsonplan/testRank.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/RankJsonPlanTest_jsonplan/testRank.out @@ -72,6 +72,9 @@ }, { "id" : 4, "type" : "stream-exec-rank_1", + "configuration" : { + "table.exec.rank.topn-cache-size" : "10000" + }, "rankType" : "ROW_NUMBER", "partition" : { "fields" : [ 1 ] @@ -126,6 +129,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/SortLimitJsonPlanTest_jsonplan/testSortLimit.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/SortLimitJsonPlanTest_jsonplan/testSortLimit.out index 6ba5bf6933d18..c0a3251360a2e 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/SortLimitJsonPlanTest_jsonplan/testSortLimit.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/SortLimitJsonPlanTest_jsonplan/testSortLimit.out @@ -58,6 +58,9 @@ }, { "id" : 3, "type" : "stream-exec-sort-limit_1", + "configuration" : { + "table.exec.rank.topn-cache-size" : "10000" + }, "orderBy" : { "fields" : [ { "index" : 1, @@ -113,6 +116,12 @@ }, { "id" : 5, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testOverwrite.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testOverwrite.out index 4dc91d95c80d1..c9d3d04531344 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testOverwrite.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testOverwrite.out @@ -34,6 +34,12 @@ }, { "id" : 2, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testPartitioning.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testPartitioning.out index 4318664e07ec7..4858fb995f639 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testPartitioning.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testPartitioning.out @@ -69,6 +69,12 @@ }, { "id" : 3, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testWritingMetadata.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testWritingMetadata.out index 00f9f79f525e2..60411d8a3d952 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testWritingMetadata.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testWritingMetadata.out @@ -34,6 +34,12 @@ }, { "id" : 2, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testFilterPushDown.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testFilterPushDown.out index 0c8aa8b0edde7..eed66e54fd254 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testFilterPushDown.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testFilterPushDown.out @@ -53,6 +53,12 @@ }, { "id" : 2, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testLimitPushDown.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testLimitPushDown.out index 55037d9632d43..50c566df575f3 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testLimitPushDown.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testLimitPushDown.out @@ -50,6 +50,9 @@ }, { "id" : 3, "type" : "stream-exec-limit_1", + "configuration" : { + "table.exec.rank.topn-cache-size" : "10000" + }, "rankRange" : { "type" : "Constant", "start" : 1, @@ -79,6 +82,12 @@ }, { "id" : 4, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testPartitionPushDown.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testPartitionPushDown.out index 772fffbcb36ec..9232850325948 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testPartitionPushDown.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testPartitionPushDown.out @@ -81,6 +81,12 @@ }, { "id" : 3, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testProjectPushDown.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testProjectPushDown.out index 9de33800c77e2..98a2518124777 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testProjectPushDown.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testProjectPushDown.out @@ -43,6 +43,12 @@ }, { "id" : 2, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`sink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testReadingMetadata.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testReadingMetadata.out index 265707e32ce50..a8014cf440021 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testReadingMetadata.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testReadingMetadata.out @@ -49,6 +49,12 @@ }, { "id" : 2, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`sink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testWatermarkPushDown.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testWatermarkPushDown.out index f8798e3788986..bf02aa522339c 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testWatermarkPushDown.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testWatermarkPushDown.out @@ -113,6 +113,12 @@ }, { "id" : 2, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`sink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalJoinJsonPlanTest_jsonplan/testJoinTemporalFunction.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalJoinJsonPlanTest_jsonplan/testJoinTemporalFunction.out index 7c4fd06033448..6c093078332da 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalJoinJsonPlanTest_jsonplan/testJoinTemporalFunction.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalJoinJsonPlanTest_jsonplan/testJoinTemporalFunction.out @@ -324,6 +324,12 @@ }, { "id" : 9, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalJoinJsonPlanTest_jsonplan/testTemporalTableJoin.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalJoinJsonPlanTest_jsonplan/testTemporalTableJoin.out index 2bf45647b143a..7666686f2669b 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalJoinJsonPlanTest_jsonplan/testTemporalTableJoin.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalJoinJsonPlanTest_jsonplan/testTemporalTableJoin.out @@ -324,6 +324,12 @@ }, { "id" : 9, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalSortJsonPlanTest_jsonplan/testSortProcessingTime.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalSortJsonPlanTest_jsonplan/testSortProcessingTime.out index 6386343f939c2..a8b9d4c857c1a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalSortJsonPlanTest_jsonplan/testSortProcessingTime.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalSortJsonPlanTest_jsonplan/testSortProcessingTime.out @@ -309,6 +309,12 @@ }, { "id" : 7, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalSortJsonPlanTest_jsonplan/testSortRowTime.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalSortJsonPlanTest_jsonplan/testSortRowTime.out index 1c5ffe17ca623..1bfbf4d324a68 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalSortJsonPlanTest_jsonplan/testSortRowTime.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalSortJsonPlanTest_jsonplan/testSortRowTime.out @@ -255,6 +255,12 @@ }, { "id" : 7, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/UnionJsonPlanTest_jsonplan/testUnion.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/UnionJsonPlanTest_jsonplan/testUnion.out index 1e050955c0a4e..246a01af6848a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/UnionJsonPlanTest_jsonplan/testUnion.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/UnionJsonPlanTest_jsonplan/testUnion.out @@ -92,6 +92,12 @@ }, { "id" : 4, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ValuesJsonPlanTest_jsonplan/testValues.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ValuesJsonPlanTest_jsonplan/testValues.out index a1a7ff0d2931c..fd83244d9dd98 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ValuesJsonPlanTest_jsonplan/testValues.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ValuesJsonPlanTest_jsonplan/testValues.out @@ -78,6 +78,12 @@ }, { "id" : 3, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WatermarkAssignerJsonPlanTest_jsonplan/testWatermarkAssigner.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WatermarkAssignerJsonPlanTest_jsonplan/testWatermarkAssigner.out index 15b2bb8e1bf7d..4ebd7ddefd6c8 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WatermarkAssignerJsonPlanTest_jsonplan/testWatermarkAssigner.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WatermarkAssignerJsonPlanTest_jsonplan/testWatermarkAssigner.out @@ -104,6 +104,12 @@ }, { "id" : 3, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`sink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testDistinctSplitEnabled.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testDistinctSplitEnabled.out index be85b4c02b87c..94423a788627f 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testDistinctSplitEnabled.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testDistinctSplitEnabled.out @@ -242,6 +242,9 @@ }, { "id" : 5, "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0, 3 ], "aggCalls" : [ { "name" : null, @@ -399,6 +402,9 @@ }, { "id" : 7, "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0, 1 ], "aggCalls" : [ { "name" : null, @@ -549,6 +555,9 @@ }, { "id" : 9, "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : null, @@ -619,6 +628,9 @@ }, { "id" : 11, "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : null, @@ -772,6 +784,12 @@ }, { "id" : 13, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindow.out index 4d3c3dcca26e0..cdfeaf552f19e 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindow.out @@ -221,6 +221,9 @@ }, { "id" : 5, "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$2", @@ -282,6 +285,9 @@ }, { "id" : 7, "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$2", @@ -407,6 +413,12 @@ }, { "id" : 9, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindowWithOffset.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindowWithOffset.out index 3915c80efb63d..aa5990662ab0b 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindowWithOffset.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindowWithOffset.out @@ -221,6 +221,9 @@ }, { "id" : 5, "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$2", @@ -283,6 +286,9 @@ }, { "id" : 7, "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$2", @@ -409,6 +415,12 @@ }, { "id" : 9, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out index 4c8d84a27a642..74a780364d821 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out @@ -221,6 +221,9 @@ }, { "id" : 5, "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$1", @@ -282,6 +285,9 @@ }, { "id" : 7, "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$1", @@ -403,6 +409,12 @@ }, { "id" : 9, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindowWithOffset.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindowWithOffset.out index 25d8c8f078067..fd02b9e0855f0 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindowWithOffset.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindowWithOffset.out @@ -221,6 +221,9 @@ }, { "id" : 5, "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$1", @@ -283,6 +286,9 @@ }, { "id" : 7, "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$1", @@ -405,6 +411,12 @@ }, { "id" : 9, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out index 6e258b3309337..9d9064218f709 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out @@ -221,6 +221,9 @@ }, { "id" : 5, "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$3", @@ -463,6 +466,9 @@ }, { "id" : 7, "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$3", @@ -619,6 +625,12 @@ }, { "id" : 9, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindowWithOffset.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindowWithOffset.out index ea0bd3702b511..577951a3fe647 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindowWithOffset.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindowWithOffset.out @@ -221,6 +221,9 @@ }, { "id" : 5, "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$3", @@ -464,6 +467,9 @@ }, { "id" : 7, "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$3", @@ -621,6 +627,12 @@ }, { "id" : 9, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeCumulateWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeCumulateWindow.out index 0217355037cfb..b7c7cb2a05c55 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeCumulateWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeCumulateWindow.out @@ -286,6 +286,9 @@ }, { "id" : 6, "type" : "stream-exec-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$1", @@ -377,6 +380,12 @@ }, { "id" : 8, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out index 2973abc7e9424..0f5cbd6d71361 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out @@ -287,6 +287,9 @@ }, { "id" : 6, "type" : "stream-exec-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$1", @@ -377,6 +380,12 @@ }, { "id" : 8, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out index 02faef6fec2ea..2b2369f3e4a15 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out @@ -266,6 +266,9 @@ }, { "id" : 6, "type" : "stream-exec-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$2", @@ -360,6 +363,12 @@ }, { "id" : 8, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowJoinJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowJoinJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out index 617e8b844b93a..2a59a3f7a2db8 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowJoinJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowJoinJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out @@ -170,6 +170,9 @@ }, { "id" : 4, "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "cnt", @@ -305,6 +308,9 @@ }, { "id" : 6, "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "cnt", @@ -650,6 +656,9 @@ }, { "id" : 12, "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "cnt", @@ -785,6 +794,9 @@ }, { "id" : 14, "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "cnt", @@ -963,6 +975,9 @@ }, { "id" : 17, "type" : "stream-exec-window-join_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "joinSpec" : { "joinType" : "INNER", "leftKeys" : [ 0 ], @@ -1064,6 +1079,12 @@ }, { "id" : 19, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testFollowedByWindowDeduplicate.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testFollowedByWindowDeduplicate.out index ea72435dd9e67..a10937b0c33f1 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testFollowedByWindowDeduplicate.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testFollowedByWindowDeduplicate.out @@ -209,6 +209,9 @@ }, { "id" : 4, "type" : "stream-exec-window-table-function_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "windowing" : { "strategy" : "TimeAttribute", "window" : { @@ -381,6 +384,9 @@ }, { "id" : 7, "type" : "stream-exec-window-deduplicate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "partitionKeys" : [ 0 ], "orderKey" : 3, "keepLastRow" : true, @@ -470,6 +476,12 @@ }, { "id" : 9, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testFollowedByWindowJoin.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testFollowedByWindowJoin.out index c98861677f0be..df66bb07961c5 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testFollowedByWindowJoin.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testFollowedByWindowJoin.out @@ -168,6 +168,9 @@ }, { "id" : 4, "type" : "stream-exec-window-table-function_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "windowing" : { "strategy" : "TimeAttribute", "window" : { @@ -453,6 +456,9 @@ }, { "id" : 10, "type" : "stream-exec-window-table-function_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "windowing" : { "strategy" : "TimeAttribute", "window" : { @@ -573,6 +579,9 @@ }, { "id" : 13, "type" : "stream-exec-window-join_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "joinSpec" : { "joinType" : "INNER", "leftKeys" : [ 2 ], @@ -674,6 +683,12 @@ }, { "id" : 15, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testFollowedByWindowRank.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testFollowedByWindowRank.out index 5242939a4bc87..398af0349871a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testFollowedByWindowRank.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testFollowedByWindowRank.out @@ -209,6 +209,9 @@ }, { "id" : 4, "type" : "stream-exec-window-table-function_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "windowing" : { "strategy" : "TimeAttribute", "window" : { @@ -323,6 +326,9 @@ }, { "id" : 7, "type" : "stream-exec-window-rank_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "rankType" : "ROW_NUMBER", "partitionSpec" : { "fields" : [ 0 ] @@ -401,6 +407,12 @@ }, { "id" : 9, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testIndividualWindowTVF.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testIndividualWindowTVF.out index aea8c5010bac0..24430239300b4 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testIndividualWindowTVF.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testIndividualWindowTVF.out @@ -168,6 +168,9 @@ }, { "id" : 4, "type" : "stream-exec-window-table-function_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "windowing" : { "strategy" : "TimeAttribute", "window" : { @@ -261,6 +264,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testIndividualWindowTVFProcessingTime.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testIndividualWindowTVFProcessingTime.out index e3af3a1fbf818..f3a32187d22fb 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testIndividualWindowTVFProcessingTime.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testIndividualWindowTVFProcessingTime.out @@ -264,6 +264,9 @@ }, { "id" : 5, "type" : "stream-exec-window-table-function_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "windowing" : { "strategy" : "TimeAttribute", "window" : { @@ -359,6 +362,12 @@ }, { "id" : 7, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRuleTest.xml index f8d4745254ba0..599e31846232c 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRuleTest.xml @@ -231,7 +231,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -250,7 +250,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -269,7 +269,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -288,7 +288,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -307,7 +307,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -326,7 +326,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -383,7 +383,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -402,7 +402,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -421,7 +421,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -478,7 +478,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.xml index 2115b4b73f2e4..9c99e35123b64 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.xml @@ -312,6 +312,23 @@ LogicalProject(id=[$0], EXPR$1=[ITEM($5, _UTF-16LE'e')]) + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleTest.xml index deab0da75fd68..ae25b30e84677 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleTest.xml @@ -111,8 +111,8 @@ LogicalProject(a=[$0], b=[$1], c=[$2], metadata=[$3], computed=[$4]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteIntersectAllRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteIntersectAllRuleTest.xml index c5a3d2ff1c104..ad832c4bab505 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteIntersectAllRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteIntersectAllRuleTest.xml @@ -43,8 +43,7 @@ LogicalProject(c=[$2]) : +- LogicalProject(f=[$0], vcol_left_marker=[null:BOOLEAN], vcol_right_marker=[true]) : +- LogicalProject(f=[$2]) : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) - +- LogicalProject(c=[$0]) - +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) f0)]) + +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType(VARCHAR(2147483647) c)]) ]]> @@ -77,8 +76,7 @@ LogicalProject(c=[$2]) : +- LogicalProject(f=[$0], vcol_left_marker=[null:BOOLEAN], vcol_right_marker=[true]) : +- LogicalProject(f=[$2]) : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) - +- LogicalProject(c=[$0]) - +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) f0)]) + +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType(VARCHAR(2147483647) c)]) ]]> @@ -113,8 +111,7 @@ LogicalProject(c=[$2]) : +- LogicalProject(d=[$0], e=[$1], f=[$2], vcol_left_marker=[null:BOOLEAN], vcol_right_marker=[true]) : +- LogicalProject(d=[$0], e=[$1], f=[$2]) : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) - +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1, $2, $3)], rowType=[RecordType:peek_no_expand(INTEGER f0, BIGINT f1, VARCHAR(2147483647) f2)]) + +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1, $2, $3)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c)]) ]]> @@ -147,8 +144,7 @@ LogicalProject(c=[$2]) : +- LogicalProject(f=[$2]) : +- LogicalFilter(condition=[=(1, 0)]) : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) - +- LogicalProject(c=[$0]) - +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) f0)]) + +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType(VARCHAR(2147483647) c)]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteMinusAllRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteMinusAllRuleTest.xml index ae6d9dc74ba1c..e8956490d1cb6 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteMinusAllRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteMinusAllRuleTest.xml @@ -43,8 +43,7 @@ LogicalProject(c=[$2]) : +- LogicalProject(f=[$0], vcol_marker=[-1:BIGINT]) : +- LogicalProject(f=[$2]) : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) - +- LogicalProject(c=[$0]) - +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) f0)]) + +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType(VARCHAR(2147483647) c)]) ]]> @@ -77,8 +76,7 @@ LogicalProject(c=[$2]) : +- LogicalProject(f=[$0], vcol_marker=[-1:BIGINT]) : +- LogicalProject(f=[$2]) : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) - +- LogicalProject(c=[$0]) - +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) f0)]) + +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType(VARCHAR(2147483647) c)]) ]]> @@ -111,8 +109,7 @@ LogicalProject(c=[$2]) : +- LogicalProject(f=[$2]) : +- LogicalFilter(condition=[=(1, 0)]) : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) - +- LogicalProject(c=[$0]) - +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) f0)]) + +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType(VARCHAR(2147483647) c)]) ]]> @@ -147,8 +144,7 @@ LogicalProject(c=[$2]) : +- LogicalProject(d=[$0], e=[$1], f=[$2], vcol_marker=[-1:BIGINT]) : +- LogicalProject(d=[$0], e=[$1], f=[$2]) : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) - +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1, $2, $3)], rowType=[RecordType:peek_no_expand(INTEGER f0, BIGINT f1, VARCHAR(2147483647) f2)]) + +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1, $2, $3)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c)]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.xml index d97e11194e1e3..b8de2fc902e7c 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.xml @@ -16,13 +16,18 @@ See the License for the specific language governing permissions and limitations under the License. --> - + @@ -30,7 +35,9 @@ FROM inventory LogicalProject(EXPR$0=[$2], name=[$0], type=[$1]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[SUM($2)]) +- LogicalProject(name=[$1], type=[$4], amount=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, inventory]]) + +- LogicalSort(fetch=[100]) + +- LogicalProject(id=[$0], name=[$1], amount=[$2], price=[$3], type=[$4]) + +- LogicalTableScan(table=[[default_catalog, default_database, inventory]]) ]]> @@ -38,14 +45,19 @@ LogicalProject(EXPR$0=[$2], name=[$0], type=[$1]) Calc(select=[EXPR$0, name, type]) +- HashAggregate(isMerge=[true], groupBy=[name, type], select=[name, type, Final_SUM(sum$0) AS EXPR$0]) +- Exchange(distribution=[hash[name, type]]) - +- TableSourceScan(table=[[default_catalog, default_database, inventory, project=[name, type, amount], metadata=[], aggregates=[grouping=[name,type], aggFunctions=[LongSumAggFunction(amount)]]]], fields=[name, type, sum$0]) + +- LocalHashAggregate(groupBy=[name, type], select=[name, type, Partial_SUM(amount) AS sum$0]) + +- Calc(select=[name, type, amount]) + +- Limit(offset=[0], fetch=[100], global=[true]) + +- Exchange(distribution=[single]) + +- Limit(offset=[0], fetch=[100], global=[false]) + +- TableSourceScan(table=[[default_catalog, default_database, inventory, limit=[100]]], fields=[id, name, amount, price, type]) ]]> - + @@ -62,109 +74,185 @@ LogicalProject(EXPR$0=[$2], name=[$0], type=[$1]) - + + + + + + + + + + + + + count(id) FILTER(WHERE id > 100), + name +FROM inventory + group by name]]> ($0, 100))]) + +- LogicalTableScan(table=[[default_catalog, default_database, inventory]]) ]]> (id, 100)) AS $f4]) + +- TableSourceScan(table=[[default_catalog, default_database, inventory, project=[name, id, amount, price], metadata=[]]], fields=[name, id, amount, price]) ]]> - + + count(id), + name +FROM inventory + group by name]]> - + + group by name]]> - + + id, + amount, + sum(price) over (partition by name), + name +FROM inventory]]> (COUNT($3) OVER (PARTITION BY $1), 0), $SUM0($3) OVER (PARTITION BY $1), null:BIGINT)], name=[$1]) ++- LogicalTableScan(table=[[default_catalog, default_database, inventory]]) ]]> (w0$o0, 0), w0$o1, null:BIGINT) AS EXPR$2, name]) ++- OverAggregate(partitionBy=[name], window#0=[COUNT(price) AS w0$o0, $SUM0(price) AS w0$o1 RANG BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], select=[id, name, amount, price, w0$o0, w0$o1]) + +- Sort(orderBy=[name ASC]) + +- Exchange(distribution=[hash[name]]) + +- TableSourceScan(table=[[default_catalog, default_database, inventory, project=[id, name, amount, price], metadata=[]]], fields=[id, name, amount, price]) +]]> + + + + + + + + + + + @@ -193,6 +281,28 @@ Calc(select=[EXPR$0, name, type]) +- HashAggregate(isMerge=[true], groupBy=[name, type], select=[name, type, Final_SUM(sum$0) AS EXPR$0]) +- Exchange(distribution=[hash[name, type]]) +- TableSourceScan(table=[[default_catalog, default_database, inventory, filter=[=(id, 123:BIGINT)], project=[name, type, amount], metadata=[], aggregates=[grouping=[name,type], aggFunctions=[LongSumAggFunction(amount)]]]], fields=[name, type, sum$0]) +]]> + + + + + + + + + + + @@ -254,46 +364,39 @@ Calc(select=[EXPR$0, type, name]) ]]> - + + min(id), + max(amount), + sum(price), + avg(price), + count(id) +FROM inventory]]> - + @@ -301,9 +404,8 @@ FROM ( LogicalProject(EXPR$0=[$2], name=[$0], type=[$1]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[SUM($2)]) +- LogicalProject(name=[$1], type=[$4], amount=[$2]) - +- LogicalSort(fetch=[100]) - +- LogicalProject(id=[$0], name=[$1], amount=[$2], price=[$3], type=[$4]) - +- LogicalTableScan(table=[[default_catalog, default_database, inventory]]) + +- LogicalFilter(condition=[=($0, 123)]) + +- LogicalTableScan(table=[[default_catalog, default_database, inventory_no_proj]]) ]]> @@ -311,188 +413,86 @@ LogicalProject(EXPR$0=[$2], name=[$0], type=[$1]) Calc(select=[EXPR$0, name, type]) +- HashAggregate(isMerge=[true], groupBy=[name, type], select=[name, type, Final_SUM(sum$0) AS EXPR$0]) +- Exchange(distribution=[hash[name, type]]) - +- LocalHashAggregate(groupBy=[name, type], select=[name, type, Partial_SUM(amount) AS sum$0]) - +- Calc(select=[name, type, amount]) - +- Limit(offset=[0], fetch=[100], global=[true]) - +- Exchange(distribution=[single]) - +- Limit(offset=[0], fetch=[100], global=[false]) - +- TableSourceScan(table=[[default_catalog, default_database, inventory, limit=[100]]], fields=[id, name, amount, price, type]) -]]> - - - - - - - - - - - - - - - - - - - - - - - + + avg(price), + count(id) +FROM inventory]]> - + + group by name, type]]> - - - - - - - - (COUNT($3) OVER (PARTITION BY $1), 0), $SUM0($3) OVER (PARTITION BY $1), null:BIGINT)], name=[$1]) -+- LogicalTableScan(table=[[default_catalog, default_database, inventory]]) -]]> - - - (w0$o0, 0), w0$o1, null:BIGINT) AS EXPR$2, name]) -+- OverAggregate(partitionBy=[name], window#0=[COUNT(price) AS w0$o0, $SUM0(price) AS w0$o1 RANG BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], select=[id, name, amount, price, w0$o0, w0$o1]) - +- Sort(orderBy=[name ASC]) - +- Exchange(distribution=[hash[name]]) - +- TableSourceScan(table=[[default_catalog, default_database, inventory, project=[id, name, amount, price], metadata=[]]], fields=[id, name, amount, price]) +Calc(select=[EXPR$0, name, type]) ++- HashAggregate(isMerge=[true], groupBy=[name, type], select=[name, type, Final_SUM(sum$0) AS EXPR$0]) + +- Exchange(distribution=[hash[name, type]]) + +- LocalHashAggregate(groupBy=[name, type], select=[name, type, Partial_SUM(amount) AS sum$0]) + +- TableSourceScan(table=[[default_catalog, default_database, inventory, project=[name, type, amount], metadata=[]]], fields=[name, type, amount]) ]]> - + 100), - name + sum(amount), + name, + type FROM inventory - group by name]]> + group by name, type]]> ($0, 100))]) +LogicalProject(EXPR$0=[$2], name=[$0], type=[$1]) ++- LogicalAggregate(group=[{0, 1}], EXPR$0=[SUM($2)]) + +- LogicalProject(name=[$1], type=[$4], amount=[$2]) +- LogicalTableScan(table=[[default_catalog, default_database, inventory]]) ]]> (id, 100)) AS $f4]) - +- TableSourceScan(table=[[default_catalog, default_database, inventory, project=[name, id, amount, price], metadata=[]]], fields=[name, id, amount, price]) +Calc(select=[EXPR$0, name, type]) ++- SortAggregate(isMerge=[true], groupBy=[name, type], select=[name, type, Final_SUM(sum$0) AS EXPR$0]) + +- Sort(orderBy=[name ASC, type ASC]) + +- Exchange(distribution=[hash[name, type]]) + +- TableSourceScan(table=[[default_catalog, default_database, inventory, project=[name, type, amount], metadata=[], aggregates=[grouping=[name,type], aggFunctions=[LongSumAggFunction(amount)]]]], fields=[name, type, sum$0]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/LegacyTableFactoryTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/LegacyTableFactoryTest.xml new file mode 100644 index 0000000000000..9993c599586e9 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/LegacyTableFactoryTest.xml @@ -0,0 +1,52 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.xml index 59d848b8a5d98..61552b112b272 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.xml @@ -58,8 +58,8 @@ LogicalIntersect(all=[true]) vcol_right_cnt), vcol_right_cnt, vcol_left_cnt) AS $f0, c], where=[((vcol_left_cnt >= 1) AND (vcol_right_cnt >= 1))]) +- GroupAggregate(groupBy=[c], select=[c, COUNT(vcol_left_marker) AS vcol_left_cnt, COUNT(vcol_right_marker) AS vcol_right_cnt]) +- Exchange(distribution=[hash[c]]) @@ -182,8 +182,8 @@ LogicalMinus(all=[true]) 0)]) +- GroupAggregate(groupBy=[c], select=[c, SUM(vcol_marker) AS sum_vcol_marker]) +- Exchange(distribution=[hash[c]]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml index 8759c6acf50af..62c3334a959cc 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml @@ -192,8 +192,8 @@ LogicalProject(a=[$0], other_metadata=[CAST($4):INTEGER], b=[$1], c=[$2], metada @@ -209,8 +209,25 @@ LogicalProject(b=[$1], other_metadata=[CAST($4):INTEGER]) + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.xml index b7dea9f32617b..2b8b78770c8c2 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.xml @@ -255,6 +255,22 @@ Sink(table=[default_catalog.default_database.sink], fields=[id, city_name]) }]]> + + + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/ColumnFunctionsTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/ColumnFunctionsTest.xml index 1c8dab3ceb709..063fcca673239 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/ColumnFunctionsTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/ColumnFunctionsTest.xml @@ -153,12 +153,12 @@ Join(joinType=[InnerJoin], where=[(int1 = int2)], select=[int1, long1, string1, diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.xml index cbee6a2a6ae08..2542e8ddb26fa 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.xml @@ -21,12 +21,12 @@ limitations under the License. @@ -63,13 +63,13 @@ LogicalProject(c=[$0], d=[$1]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], rowType=[RecordType(VARCHAR(2147483647) d, INTEGER e)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], rowType=[RecordType(VARCHAR(2147483647) d, INTEGER e)]) ]]> ($1, 20)]) ++- Correlate(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], correlate=[table(*org.apache.flink.table.planner.utils.TableFunc0*(c))], select=[a,b,c,d,e], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) d, INTEGER e)], joinType=[INNER], condition=[>($1, 20)]) +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -84,13 +84,13 @@ LogicalProject(c=[$0], d=[$1]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], rowType=[RecordType(VARCHAR(2147483647) d, INTEGER e)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], rowType=[RecordType(VARCHAR(2147483647) d, INTEGER e)]) ]]> ($1, 20)]) ++- Correlate(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], correlate=[table(*org.apache.flink.table.planner.utils.TableFunc0*(c))], select=[a,b,c,d,e], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) d, INTEGER e)], joinType=[INNER], condition=[>($1, 20)]) +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -101,13 +101,13 @@ Calc(select=[c, d]) LogicalProject(c=[$2], s=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)]) ]]> @@ -118,13 +118,13 @@ Calc(select=[c, s]) LogicalProject(c=[$2], s=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2, _UTF-16LE'$')], rowType=[RecordType(VARCHAR(2147483647) s)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2, _UTF-16LE'$')], rowType=[RecordType(VARCHAR(2147483647) s)]) ]]> @@ -135,13 +135,13 @@ Calc(select=[c, s]) LogicalProject(c=[$2], name=[$3], len=[$4]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc2*(*org.apache.flink.table.planner.expressions.utils.Func13$aceadf1af6c698a4705a8fbd3984d0a3*($2))], rowType=[RecordType(VARCHAR(2147483647) name, INTEGER len)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc2*(*org.apache.flink.table.planner.expressions.utils.Func13$aceadf1af6c698a4705a8fbd3984d0a3*($2))], rowType=[RecordType(VARCHAR(2147483647) name, INTEGER len)]) ]]> @@ -153,13 +153,13 @@ LogicalFilter(condition=[>($2, 2)]) +- LogicalProject(c=[$2], name=[$3], len=[$4]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc2*($2)], rowType=[RecordType(VARCHAR(2147483647) name, INTEGER len)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc2*($2)], rowType=[RecordType(VARCHAR(2147483647) name, INTEGER len)]) ]]> ($1, 2)]) ++- Correlate(invocation=[*org.apache.flink.table.planner.utils.TableFunc2*($2)], correlate=[table(*org.apache.flink.table.planner.utils.TableFunc2*(c))], select=[a,b,c,name,len], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) name, INTEGER len)], joinType=[INNER], condition=[>($1, 2)]) +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -170,14 +170,13 @@ Calc(select=[c, name, len]) LogicalProject(f0=[AS($3, _UTF-16LE'f0')], f1=[AS($4, _UTF-16LE'f1')]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(f1, f2, f3)]]]) - +- LogicalProject(f0=[$0], f1_0=[$1]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc2*($2)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) f0, INTEGER f1)]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc2*($2)], rowType=[RecordType(VARCHAR(2147483647) f0, INTEGER f1_0)]) ]]> @@ -209,12 +208,12 @@ Correlate(invocation=[str_split(_UTF-16LE'Jack,John')], correlate=[table(str_spl @@ -291,13 +290,13 @@ Correlate(invocation=[str_split(_UTF-16LE'Jack,John', _UTF-16LE',')], correlate= LogicalProject(c=[$2], s=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[left], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)]) ]]> @@ -307,12 +306,12 @@ Calc(select=[c, s]) @@ -322,12 +321,12 @@ Correlate(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*(SUBSTRIN diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TemporalTableFunctionJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TemporalTableFunctionJoinTest.xml index 0525902625f69..47f118e34d5cf 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TemporalTableFunctionJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TemporalTableFunctionJoinTest.xml @@ -25,7 +25,7 @@ LogicalJoin(condition=[=($3, $1)], joinType=[inner]) : +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) : :- LogicalProject(o_rowtime=[AS($0, _UTF-16LE'o_rowtime')], o_comment=[AS($1, _UTF-16LE'o_comment')], o_amount=[AS($2, _UTF-16LE'o_amount')], o_currency=[AS($3, _UTF-16LE'o_currency')], o_secondary_key=[AS($4, _UTF-16LE'o_secondary_key')]) : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) -: +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.functions.TemporalTableFunctionImpl$bbf912e58a3fb2d083552961c0f87dbe*($0)], rowType=[RecordType(TIMESTAMP(3) *ROWTIME* rowtime, VARCHAR(2147483647) comment, VARCHAR(2147483647) currency, INTEGER rate, INTEGER secondary_key)], elementType=[class [Ljava.lang.Object;]) +: +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.functions.TemporalTableFunctionImpl$bbf912e58a3fb2d083552961c0f87dbe*($0)], rowType=[RecordType(TIMESTAMP(3) *ROWTIME* rowtime, VARCHAR(2147483647) comment, VARCHAR(2147483647) currency, INTEGER rate, INTEGER secondary_key)]) +- LogicalTableScan(table=[[default_catalog, default_database, ThirdTable]]) ]]> @@ -53,7 +53,7 @@ LogicalProject(rate=[AS(*($0, $4), _UTF-16LE'rate')]) +- LogicalFilter(condition=[=($3, $1)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.functions.TemporalTableFunctionImpl$225833b9896cb7740aed9150b8cc7fd9*($2)], rowType=[RecordType(VARCHAR(2147483647) currency, INTEGER rate, TIMESTAMP(3) *ROWTIME* rowtime)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.functions.TemporalTableFunctionImpl$225833b9896cb7740aed9150b8cc7fd9*($2)], rowType=[RecordType(VARCHAR(2147483647) currency, INTEGER rate, TIMESTAMP(3) *ROWTIME* rowtime)]) ]]> @@ -74,7 +74,7 @@ LogicalProject(rate=[AS(*($0, $4), _UTF-16LE'rate')]) +- LogicalFilter(condition=[=($3, $1)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - +- LogicalTableFunctionScan(invocation=[Rates($2)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) currency, INTEGER rate, TIMESTAMP(3) *ROWTIME* rowtime)]) + +- LogicalTableFunctionScan(invocation=[Rates($2)], rowType=[RecordType(VARCHAR(2147483647) currency, INTEGER rate, TIMESTAMP(3) *ROWTIME* rowtime)]) ]]> @@ -95,7 +95,7 @@ LogicalProject(rate=[AS(*($0, $4), _UTF-16LE'rate')]) +- LogicalFilter(condition=[=($3, $1)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, ProctimeOrders]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.functions.TemporalTableFunctionImpl$170cc46c47df69784f267e43f61e8e9d*($2)], rowType=[RecordType(VARCHAR(2147483647) currency, INTEGER rate, TIMESTAMP_LTZ(3) *PROCTIME* proctime)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.functions.TemporalTableFunctionImpl$170cc46c47df69784f267e43f61e8e9d*($2)], rowType=[RecordType(VARCHAR(2147483647) currency, INTEGER rate, TIMESTAMP_LTZ(3) *PROCTIME* proctime)]) ]]> @@ -117,7 +117,7 @@ LogicalProject(rate=[AS(*($0, $4), _UTF-16LE'rate')]) +- LogicalFilter(condition=[=($3, $1)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.functions.TemporalTableFunctionImpl$225833b9896cb7740aed9150b8cc7fd9*($2)], rowType=[RecordType(VARCHAR(2147483647) currency, INTEGER rate, TIMESTAMP(3) *ROWTIME* rowtime)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.functions.TemporalTableFunctionImpl$225833b9896cb7740aed9150b8cc7fd9*($2)], rowType=[RecordType(VARCHAR(2147483647) currency, INTEGER rate, TIMESTAMP(3) *ROWTIME* rowtime)]) ]]> diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala index 3f9c9067a2028..e1459435b14b0 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala @@ -21,7 +21,7 @@ package org.apache.flink.table.api import org.apache.flink.api.common.RuntimeExecutionMode import org.apache.flink.api.common.typeinfo.Types.STRING import org.apache.flink.api.scala._ -import org.apache.flink.configuration.{Configuration, ExecutionOptions} +import org.apache.flink.configuration.{Configuration, CoreOptions, ExecutionOptions} import org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches import org.apache.flink.streaming.api.environment.LocalStreamEnvironment import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment @@ -142,11 +142,11 @@ class TableEnvironmentTest { @Test def testStreamTableEnvironmentExecutionExplainWithConfParallelism(): Unit = { val execEnv = StreamExecutionEnvironment.getExecutionEnvironment - val settings = EnvironmentSettings.newInstance().inStreamingMode().build() - val tEnv = StreamTableEnvironment.create(execEnv, settings) val configuration = new Configuration() - configuration.setInteger("parallelism.default", 4) - tEnv.getConfig.addConfiguration(configuration) + configuration.set(CoreOptions.DEFAULT_PARALLELISM, Integer.valueOf(4)) + val settings = + EnvironmentSettings.newInstance().inStreamingMode().withConfiguration(configuration).build() + val tEnv = StreamTableEnvironment.create(execEnv, settings) verifyTableEnvironmentExecutionExplain(tEnv) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/calcite/CalciteConfigBuilderTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/calcite/CalciteConfigBuilderTest.scala index 88f1b6bc5de3d..38b4a6311a669 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/calcite/CalciteConfigBuilderTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/calcite/CalciteConfigBuilderTest.scala @@ -38,7 +38,7 @@ class CalciteConfigBuilderTest { assertTrue(cc.getStreamProgram.isEmpty) val builder = new CalciteConfigBuilder() - val streamPrograms = FlinkStreamProgram.buildProgram(TableConfig.getDefault.getConfiguration) + val streamPrograms = FlinkStreamProgram.buildProgram(TableConfig.getDefault) streamPrograms.remove(FlinkStreamProgram.PHYSICAL) builder.replaceStreamProgram(streamPrograms) @@ -174,7 +174,7 @@ class CalciteConfigBuilderTest { assertTrue(config.getSqlParserConfig.isEmpty) assertTrue(config.getSqlToRelConverterConfig.isEmpty) - val streamPrograms = FlinkStreamProgram.buildProgram(TableConfig.getDefault.getConfiguration) + val streamPrograms = FlinkStreamProgram.buildProgram(TableConfig.getDefault) streamPrograms.remove(FlinkStreamProgram.PHYSICAL) builder.replaceStreamProgram(streamPrograms) val baseConfig1 = builder.build() diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala index ccdb37b7c93e4..490551ad7572e 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala @@ -4190,7 +4190,7 @@ class ScalarFunctionsTest extends ScalarTypesTestBase { // the answer BINARY will cast to STRING in ExpressionTestBase.scala testSqlApi( "IF(f7 < 5, f53, f54)", - "68656c6c6f20776f726c64") // hello world + "hello world") // hello world // test DATE, DATE testSqlApi( @@ -4398,6 +4398,6 @@ class ScalarFunctionsTest extends ScalarTypesTestBase { testSqlApi(s"IFNULL(CAST(INTERVAL '2' DAY AS VARCHAR(20)), $str2)", "+2 00:00:00.000") testSqlApi( s"IFNULL(CAST(f53 AS VARCHAR(100)), $str2)", - "68656c6c6f20776f726c64") + "hello world") } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ScalarOperatorsTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ScalarOperatorsTest.scala index 459ded7797876..899a80cfabcc7 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ScalarOperatorsTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ScalarOperatorsTest.scala @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.expressions +import org.apache.flink.table.api.{DataTypes, LiteralStringExpression, UnresolvedFieldExpression} import org.apache.flink.table.planner.expressions.utils.ScalarOperatorsTestBase import org.junit.Test @@ -239,4 +240,16 @@ class ScalarOperatorsTest extends ScalarOperatorsTestBase { "FALSE") testSqlApi("uuid() = cast(f22 as timestamp_ltz)", "NULL") } + + @Test + def testTryCast(): Unit = { + testAllApis( + "non-numeric".tryCast(DataTypes.BIGINT()), + "TRY_CAST ('non-numeric' AS BIGINT)", + "NULL") + testAllApis( + 'f10.tryCast(DataTypes.BIGINT()), + "TRY_CAST (f10 AS BIGINT)", + "NULL") + } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala index 07dff77f4a4fa..52fc78bf5fcb6 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala @@ -22,11 +22,11 @@ import org.apache.flink.api.common.TaskInfo import org.apache.flink.api.common.functions.util.RuntimeUDFContext import org.apache.flink.api.common.functions.{MapFunction, RichFunction, RichMapFunction} import org.apache.flink.api.java.typeutils.RowTypeInfo -import org.apache.flink.configuration.Configuration +import org.apache.flink.configuration.{ConfigOption, Configuration} import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import org.apache.flink.table.api import org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl -import org.apache.flink.table.api.config.ExecutionConfigOptions +import org.apache.flink.table.api.config.{ExecutionConfigOptions, TableConfigOptions} import org.apache.flink.table.api.{EnvironmentSettings, TableConfig, TableException, ValidationException} import org.apache.flink.table.data.RowData import org.apache.flink.table.data.binary.BinaryRowData @@ -38,6 +38,7 @@ import org.apache.flink.table.expressions.Expression import org.apache.flink.table.functions.ScalarFunction import org.apache.flink.table.planner.codegen.{CodeGeneratorContext, ExprCodeGenerator, FunctionCodeGenerator} import org.apache.flink.table.planner.delegation.PlannerBase +import org.apache.flink.table.planner.utils.TestingTableEnvironment import org.apache.flink.table.runtime.generated.GeneratedFunction import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType import org.apache.flink.table.types.AbstractDataType @@ -51,10 +52,12 @@ import org.apache.calcite.rel.logical.LogicalCalc import org.apache.calcite.rel.rules._ import org.apache.calcite.rex.RexNode import org.apache.calcite.sql.`type`.SqlTypeName.VARCHAR + import org.junit.Assert.{assertEquals, assertTrue, fail} import org.junit.rules.ExpectedException import org.junit.{After, Before, Rule} +import java.time.ZoneId import java.util.Collections import scala.collection.JavaConverters._ @@ -62,8 +65,6 @@ import scala.collection.mutable abstract class ExpressionTestBase { - val config = new TableConfig() - // (originalExpr, optimizedExpr, expectedResult) private val validExprs = mutable.ArrayBuffer[(String, RexNode, String)]() // (originalSqlExpr, keywords, exceptionClass) @@ -73,11 +74,14 @@ abstract class ExpressionTestBase { .ArrayBuffer[(Expression, String, Class[_ <: Throwable])]() private val env = StreamExecutionEnvironment.createLocalEnvironment(4) - private val setting = EnvironmentSettings.newInstance().inStreamingMode().build() + private val settings = EnvironmentSettings.newInstance().inStreamingMode().build() // use impl class instead of interface class to avoid // "Static methods in interface require -target:jvm-1.8" - private val tEnv = StreamTableEnvironmentImpl.create(env, setting, config) + private val tEnv = StreamTableEnvironmentImpl.create(env, settings) .asInstanceOf[StreamTableEnvironmentImpl] + + val config = tEnv.getConfig + private val resolvedDataType = if (containsLegacyTypes) { TypeConversions.fromLegacyInfoToDataType(typeInfo) } else { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/CalcTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/CalcTest.scala index cdd34e22df15b..c72f013922369 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/CalcTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/CalcTest.scala @@ -80,6 +80,12 @@ class CalcTest extends TableTestBase { util.verifyExecPlan(sql) } + @Test + def testInNonConstantValues(): Unit = { + val sql = s"SELECT * FROM MyTable WHERE b IN (1, 3, CAST(a AS INT), 5, 6) AND c = 'xx'" + util.verifyExecPlan(sql) + } + @Test def testNotIn(): Unit = { val sql = s"SELECT * FROM MyTable WHERE b NOT IN (1, 3, 4, 5, 6) OR c = 'xx'" diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.scala index 56a0a88468bfb..9f4c18f4eafd2 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.scala @@ -38,10 +38,10 @@ class DeadlockBreakupTest extends TableTestBase { @Test def testSubplanReuse_SetExchangeAsBatch(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(true)) val sqlQuery = """ |WITH t AS (SELECT x.a AS a, x.b AS b, y.d AS d, y.e AS e FROM x, y WHERE x.a = y.d) @@ -52,11 +52,11 @@ class DeadlockBreakupTest extends TableTestBase { @Test def testSubplanReuse_AddExchangeAsBatch_HashJoin(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") val sqlQuery = """ @@ -68,11 +68,11 @@ class DeadlockBreakupTest extends TableTestBase { @Test def testSubplanReuse_AddExchangeAsBatch_NestedLoopJoin(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false) - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false)) + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin") val sqlQuery = """ @@ -84,12 +84,12 @@ class DeadlockBreakupTest extends TableTestBase { @Test def testSubplanReuse_SetExchangeAsBatch_SortMergeJoin(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,HashAgg") - util.tableEnv.getConfig.getConfiguration.setLong( - OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1)) val sqlQuery = """ | @@ -104,13 +104,13 @@ class DeadlockBreakupTest extends TableTestBase { @Test def testSubplanReuse_AddExchangeAsBatch_BuildLeftSemiHashJoin(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.setLong( - OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1) - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1)) + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") val sqlQuery = """ @@ -124,8 +124,8 @@ class DeadlockBreakupTest extends TableTestBase { @Test def testSubplanReuse_SetExchangeAsBatch_OverAgg(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true)) val sqlQuery = """ |WITH r1 AS (SELECT SUM(a) OVER (PARTITION BY b ORDER BY b) AS a, b, c FROM x), @@ -141,11 +141,11 @@ class DeadlockBreakupTest extends TableTestBase { @Test def testReusedNodeIsBarrierNode(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false) - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false)) + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin,SortAgg") val sqlQuery = """ @@ -157,35 +157,35 @@ class DeadlockBreakupTest extends TableTestBase { @Test def testDataStreamReuse_SetExchangeAsBatch(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false) - util.tableEnv.getConfig.getConfiguration.setLong( - OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(false)) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1)) val sqlQuery = "SELECT * FROM t t1, t t2 WHERE t1.a = t2.a AND t1.b > 10 AND t2.c LIKE 'Test%'" util.verifyExecPlan(sqlQuery) } @Test def testDataStreamReuse_AddExchangeAsBatch_NestedLoopJoin(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(false)) val sqlQuery = "SELECT * FROM t t1, t t2 WHERE t1.a = t2.b" util.verifyExecPlan(sqlQuery) } @Test def testDataStreamReuse_AddExchangeAsBatch_HashJoin(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(false)) val sqlQuery = "SELECT * FROM t INTERSECT SELECT * FROM t" util.verifyExecPlan(sqlQuery) } @Test def testSubplanReuse_BuildAndProbeNoCommonSuccessors_HashJoin(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg") val sqlQuery = s""" @@ -202,9 +202,9 @@ class DeadlockBreakupTest extends TableTestBase { @Test def testSubplanReuse_AddSingletonExchange(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin,HashAgg") val sqlQuery = s""" @@ -219,9 +219,9 @@ class DeadlockBreakupTest extends TableTestBase { @Test def testSubplanReuse_DeadlockCausedByReusingExchange(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") val sqlQuery = s""" @@ -234,11 +234,11 @@ class DeadlockBreakupTest extends TableTestBase { @Test def testSubplanReuse_DeadlockCausedByReusingExchangeInAncestor(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_MULTIPLE_INPUT_ENABLED, false) - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_MULTIPLE_INPUT_ENABLED, Boolean.box(false)) + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") val sqlQuery = """ diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LegacySinkTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LegacySinkTest.scala index 8266cda221a40..7183c92cb5c78 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LegacySinkTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LegacySinkTest.scala @@ -46,8 +46,9 @@ class LegacySinkTest extends TableTestBase { @Test def testMultiSinks(): Unit = { val stmtSet = util.tableEnv.createStatementSet() - util.tableEnv.getConfig.getConfiguration.setBoolean( - RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true) + util.tableEnv.getConfig.set( + RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, + Boolean.box(true)) val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS sum_a, c FROM MyTable GROUP BY c") util.tableEnv.registerTable("table1", table1) val table2 = util.tableEnv.sqlQuery("SELECT SUM(sum_a) AS total_sum FROM table1") diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/MultipleInputCreationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/MultipleInputCreationTest.scala index c4f2bbb88b7c9..90024e898953c 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/MultipleInputCreationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/MultipleInputCreationTest.scala @@ -44,12 +44,12 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest util.addTableSource[(Int, Long, String, Int)]("y", 'd, 'e, 'f, 'ny) util.addTableSource[(Int, Long, String, Int)]("z", 'g, 'h, 'i, 'nz) util.addDataStream[(Int, Long, String)]("t", 'a, 'b, 'c) - util.tableConfig.getConfiguration.set(ExecutionOptions.BATCH_SHUFFLE_MODE, shuffleMode) + util.tableConfig.set(ExecutionOptions.BATCH_SHUFFLE_MODE, shuffleMode) } @Test def testBasicMultipleInput(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") val sql = """ @@ -71,9 +71,9 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest // \-> [J -> J] -> [Agg -> J -/ // | | | // y t y - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false) - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false)) + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin,SortAgg") val sql = """ @@ -102,7 +102,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest @Test def testJoinWithAggAsProbe(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg") val sql = """ @@ -119,7 +119,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest @Test def testKeepMultipleInputWithOneMemberForChainableSource(): Unit = { createChainableTableSource() - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin") val sql = "SELECT * FROM chainable LEFT JOIN x ON chainable.a = x.a" util.verifyExecPlan(sql) @@ -127,7 +127,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest @Test def testAvoidIncludingUnionFromInputSide(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin") val sql = """ @@ -141,7 +141,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest @Test def testIncludeUnionForChainableSource(): Unit = { createChainableTableSource() - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin") val sql = """ @@ -154,7 +154,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest @Test def testAvoidIncludingCalcAfterNonChainableSource(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin") val sql = """ @@ -169,7 +169,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest @Test def testIncludeCalcForChainableSource(): Unit = { createChainableTableSource() - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin") val sql = """ @@ -183,7 +183,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest @Test def testAvoidIncludingSingleton(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin,HashAgg") val sql = """ @@ -201,7 +201,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest @Test def testNoPriorityConstraint(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin") val sql = """ @@ -214,7 +214,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest @Test def testRelatedInputs(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin") val sql = """ @@ -232,7 +232,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest @Test def testRelatedInputsWithAgg(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin,SortAgg") val sql = """ @@ -250,9 +250,9 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest @Test def testRemoveRedundantUnion(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false) - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false)) + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg") val sql = """ @@ -271,9 +271,9 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest @Test def testRemoveOneInputOperatorFromRoot(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false) - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false)) + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") val sql = """ @@ -290,7 +290,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest @Test def testCleanUpMultipleInputWithOneMember(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") val sql = """ @@ -307,9 +307,9 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest @Test def testKeepUsefulUnion(): Unit = { createChainableTableSource() - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin") val sql = """ @@ -326,9 +326,9 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest @Test def testDeadlockCausedByExchangeInAncestor(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") val sql = """ diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveCollationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveCollationTest.scala index ad1f32b4fa3c4..0038f520b1989 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveCollationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveCollationTest.scala @@ -59,13 +59,13 @@ class RemoveCollationTest extends TableTestBase { FlinkStatistic.builder().tableStats(new TableStats(100L)).build() ) - util.tableEnv.getConfig.getConfiguration.setBoolean( - BatchPhysicalSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true) + util.tableEnv.getConfig.set( + BatchPhysicalSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, Boolean.box(true)) } @Test def testRemoveCollation_OverWindowAgg(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,HashAgg") val sqlQuery = """ @@ -81,7 +81,7 @@ class RemoveCollationTest extends TableTestBase { @Test def testRemoveCollation_Aggregate(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin") val sqlQuery = """ @@ -93,7 +93,7 @@ class RemoveCollationTest extends TableTestBase { @Test def testRemoveCollation_Aggregate_1(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin") val sqlQuery = """ @@ -105,7 +105,7 @@ class RemoveCollationTest extends TableTestBase { @Test def testRemoveCollation_Sort(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, true) + util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(true)) val sqlQuery = """ |WITH r AS (SELECT a, b, COUNT(c) AS cnt FROM x GROUP BY a, b) @@ -117,9 +117,9 @@ class RemoveCollationTest extends TableTestBase { @Test def testRemoveCollation_Aggregate_3(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg") - util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, true) + util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(true)) val sqlQuery = """ |WITH r AS (SELECT * FROM x ORDER BY a, b) @@ -130,7 +130,7 @@ class RemoveCollationTest extends TableTestBase { @Test def testRemoveCollation_Rank_1(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg") val sqlQuery = """ @@ -145,7 +145,7 @@ class RemoveCollationTest extends TableTestBase { @Test def testRemoveCollation_Rank_2(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg") val sqlQuery = """ @@ -174,7 +174,7 @@ class RemoveCollationTest extends TableTestBase { @Test def testRemoveCollation_Rank_4(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg") val sqlQuery = """ @@ -189,7 +189,7 @@ class RemoveCollationTest extends TableTestBase { @Test def testRemoveCollation_Rank_Singleton(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg") val sqlQuery = """ @@ -204,7 +204,7 @@ class RemoveCollationTest extends TableTestBase { @Test def testRemoveCollation_MultipleSortMergeJoins1(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin") val sql = @@ -220,7 +220,7 @@ class RemoveCollationTest extends TableTestBase { @Test def testRemoveCollation_MultipleSortMergeJoins_MultiJoinKeys1(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin") val sql = @@ -236,7 +236,7 @@ class RemoveCollationTest extends TableTestBase { @Test def testRemoveCollation_MultipleSortMergeJoins2(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin") val sql = @@ -252,7 +252,7 @@ class RemoveCollationTest extends TableTestBase { @Test def testRemoveCollation_MultipleSortMergeJoins_MultiJoinKeys2(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin") val sql = @@ -268,7 +268,7 @@ class RemoveCollationTest extends TableTestBase { @Test def testRemoveCollation_MultipleSortMergeJoins3(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin") util.addTableSource("tb1", Array[TypeInformation[_]]( @@ -344,7 +344,7 @@ class RemoveCollationTest extends TableTestBase { @Test def testRemoveCollation_Correlate1(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin,HashAgg") util.addFunction("split", new TableFunc1) val sqlQuery = @@ -358,7 +358,7 @@ class RemoveCollationTest extends TableTestBase { @Test def testRemoveCollation_Correlate2(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin,HashAgg") util.addFunction("split", new TableFunc1) val sqlQuery = @@ -373,7 +373,7 @@ class RemoveCollationTest extends TableTestBase { @Test def testRemoveCollation_Correlate3(): Unit = { // do not remove shuffle - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin,HashAgg") util.addFunction("split", new TableFunc1) val sqlQuery = diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveShuffleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveShuffleTest.scala index 39951be55ed1b..6d0bb657bc071 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveShuffleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveShuffleTest.scala @@ -45,13 +45,13 @@ class RemoveShuffleTest extends TableTestBase { Array("d", "e", "f"), FlinkStatistic.builder().tableStats(new TableStats(100L)).build() ) - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(false)) } @Test def testRemoveHashShuffle_OverWindowAgg(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg") val sqlQuery = """ @@ -68,7 +68,7 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_MultiOverWindowAgg(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg") val sqlQuery = """ @@ -85,10 +85,10 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_OverWindowAgg_PartialKey(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg") - util.tableEnv.getConfig.getConfiguration.setBoolean( - BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, true) + util.tableEnv.getConfig.set( + BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, Boolean.box(true)) // push down HashExchange[c] into HashAgg val sqlQuery = """ @@ -105,10 +105,10 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_Agg_PartialKey(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg") - util.tableEnv.getConfig.getConfiguration.setBoolean( - BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, true) + util.tableEnv.getConfig.set( + BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, Boolean.box(true)) // push down HashExchange[c] into HashAgg val sqlQuery = """ @@ -120,11 +120,11 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_HashAggregate(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg") // disable BroadcastHashJoin - util.tableEnv.getConfig.getConfiguration.setLong( - OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1)) val sqlQuery = """ |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%') @@ -135,11 +135,11 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_HashAggregate_1(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg") // disable BroadcastHashJoin - util.tableEnv.getConfig.getConfiguration.setLong( - OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1)) val sqlQuery = """ |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%') @@ -150,11 +150,11 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_HashAggregate_2(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg") // disable BroadcastHashJoin - util.tableEnv.getConfig.getConfiguration.setLong( - OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1)) val sqlQuery = """ |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%') @@ -165,11 +165,11 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_SortAggregate(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,HashAgg") // disable BroadcastHashJoin - util.tableEnv.getConfig.getConfiguration.setLong( - OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1)) val sqlQuery = """ |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%') @@ -180,11 +180,11 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_SortAggregate_1(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,HashAgg") // disable BroadcastHashJoin - util.tableEnv.getConfig.getConfiguration.setLong( - OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1)) val sqlQuery = """ |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%') @@ -195,11 +195,11 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_SortAggregate_2(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,HashAgg") // disable BroadcastHashJoin - util.tableEnv.getConfig.getConfiguration.setLong( - OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1)) val sqlQuery = """ |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%') @@ -210,10 +210,10 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_SortMergeJoin(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin") - util.tableEnv.getConfig.getConfiguration.setBoolean( - BatchPhysicalSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true) + util.tableEnv.getConfig.set( + BatchPhysicalSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, Boolean.box(true)) val sqlQuery = """ |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%') @@ -224,10 +224,10 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_SortMergeJoin_LOJ(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin") - util.tableEnv.getConfig.getConfiguration.setBoolean( - BatchPhysicalSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true) + util.tableEnv.getConfig.set( + BatchPhysicalSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, Boolean.box(true)) val sqlQuery = """ |WITH r AS (SELECT * FROM x left join (SELECT * FROM y WHERE e = 2) r on a = d) @@ -238,10 +238,10 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_SortMergeJoin_ROJ(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin") - util.tableEnv.getConfig.getConfiguration.setBoolean( - BatchPhysicalSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true) + util.tableEnv.getConfig.set( + BatchPhysicalSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, Boolean.box(true)) val sqlQuery = """ |WITH r AS (SELECT * FROM x right join (SELECT * FROM y WHERE e = 2) r on a = d) @@ -252,7 +252,7 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_SortMergeJoin_FOJ(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin") val sqlQuery = """ @@ -264,11 +264,11 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_HashJoin(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") // disable BroadcastHashJoin - util.tableEnv.getConfig.getConfiguration.setLong( - OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1)) val sqlQuery = """ |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%') @@ -279,7 +279,7 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_BroadcastHashJoin(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") val sqlQuery = """ @@ -291,11 +291,11 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_HashJoin_LOJ(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") // disable BroadcastHashJoin - util.tableEnv.getConfig.getConfiguration.setLong( - OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1)) val sqlQuery = """ |WITH r AS (SELECT * FROM x left join (SELECT * FROM y WHERE e = 2) r on a = d) @@ -306,11 +306,11 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_HashJoin_ROJ(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") // disable BroadcastHashJoin - util.tableEnv.getConfig.getConfiguration.setLong( - OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1)) val sqlQuery = """ |WITH r AS (SELECT * FROM x right join (SELECT * FROM y WHERE e = 2) r on a = d) @@ -321,11 +321,11 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_HashJoin_FOJ(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") // disable BroadcastHashJoin - util.tableEnv.getConfig.getConfiguration.setLong( - OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1)) val sqlQuery = """ |WITH r AS (SELECT * FROM x full join (SELECT * FROM y WHERE e = 2) r on a = d) @@ -336,11 +336,11 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_HashJoin_1(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") // disable BroadcastHashJoin - util.tableEnv.getConfig.getConfiguration.setLong( - OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1)) val sqlQuery = """ |WITH r1 AS (SELECT a, c, sum(b) FROM x group by a, c), @@ -352,7 +352,7 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_NestedLoopJoin(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin") val sqlQuery = """ @@ -364,13 +364,13 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_Join_PartialKey(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg") // disable BroadcastHashJoin - util.tableEnv.getConfig.getConfiguration.setLong( - OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1) - util.tableEnv.getConfig.getConfiguration.setBoolean( - BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, true) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1)) + util.tableEnv.getConfig.set( + BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, Boolean.box(true)) val sqlQuery = """ |WITH r AS (SELECT d, count(f) as cnt FROM y GROUP BY d) @@ -387,7 +387,7 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_Union(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg") val sqlQuery = """ @@ -402,7 +402,7 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_Rank(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg") val sqlQuery = """ @@ -417,10 +417,10 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_Rank_PartialKey1(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg") - util.tableEnv.getConfig.getConfiguration.setBoolean( - BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, true) + util.tableEnv.getConfig.set( + BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, Boolean.box(true)) val sqlQuery = """ |SELECT a, SUM(b) FROM ( @@ -434,10 +434,10 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_Rank_PartialKey2(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg") - util.tableEnv.getConfig.getConfiguration.setBoolean( - BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, false) + util.tableEnv.getConfig.set( + BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, Boolean.box(false)) val sqlQuery = """ |SELECT * FROM ( @@ -451,10 +451,10 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_Rank_PartialKey3(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg") - util.tableEnv.getConfig.getConfiguration.setBoolean( - BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, true) + util.tableEnv.getConfig.set( + BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, Boolean.box(true)) val sqlQuery = """ |SELECT * FROM ( @@ -468,7 +468,7 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_Rank_Singleton1(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg") val sqlQuery = """ @@ -483,7 +483,7 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_Rank_Singleton2(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg") val sqlQuery = """ @@ -498,11 +498,11 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_Correlate1(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg") // disable BroadcastHashJoin - util.tableEnv.getConfig.getConfiguration.setLong( - OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1)) util.addFunction("split", new TableFunc1) val sqlQuery = """ @@ -515,11 +515,11 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_Correlate2(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg") // disable BroadcastHashJoin - util.tableEnv.getConfig.getConfiguration.setLong( - OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1)) util.addFunction("split", new TableFunc1) val sqlQuery = """ @@ -533,11 +533,11 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveHashShuffle_Correlate3(): Unit = { // do not remove shuffle - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg") // disable BroadcastHashJoin - util.tableEnv.getConfig.getConfiguration.setLong( - OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1)) util.addFunction("split", new TableFunc1) val sqlQuery = """ @@ -555,14 +555,14 @@ class RemoveShuffleTest extends TableTestBase { @Test def testRemoveSingletonShuffle_HashAgg(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg") util.verifyExecPlan("SELECT MAX(b) FROM (SELECT SUM(b) AS b FROM x)") } @Test def testRemoveSingletonShuffle_SortAgg(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg") util.verifyExecPlan("SELECT MAX(b) FROM (SELECT SUM(b) AS b FROM x)") } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.scala index 5bf46390f74d9..b5b7abee17365 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.scala @@ -34,7 +34,7 @@ class SetOperatorsTest extends TableTestBase { @Before def before(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg") util.addTableSource[(Int, Long, String)]("T1", 'a, 'b, 'c) util.addTableSource[(Int, Long, String)]("T2", 'd, 'e, 'f) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.scala index 070de14ddd0ac..8cf961dccfb21 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.scala @@ -30,66 +30,66 @@ class SortLimitTest extends TableTestBase { private val util = batchTestUtil() util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) - util.tableEnv.getConfig.getConfiguration.setInteger( - ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, 200) + util.tableEnv.getConfig.set( + ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, Integer.valueOf(200)) @Test def testNonRangeSortWithoutOffset(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, false) + util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(false)) util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC LIMIT 5") } @Test def testNonRangeSortWithLimit0(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, false) + util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(false)) util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC LIMIT 0") } @Test def testNonRangeSortOnlyWithOffset(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, false) + util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(false)) util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC OFFSET 10 ROWS") } @Test def testNoneRangeSortWithOffsetLimit(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, false) + util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(false)) util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC, b LIMIT 10 OFFSET 1") } @Test def testNoneRangeSortWithOffsetLimit0(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, false) + util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(false)) util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC, b LIMIT 0 OFFSET 1") } @Test def testRangeSortOnWithoutOffset(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, true) + util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(true)) util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC LIMIT 5") } @Test def testRangeSortOnWithLimit0(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, true) + util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(true)) util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC LIMIT 0") } @Test def testRangeSortOnlyWithOffset(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, true) + util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(true)) util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC OFFSET 10 ROWS") } @Test def testRangeSortWithOffsetLimit(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, true) + util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(true)) util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC, b LIMIT 10 OFFSET 1") } @Test def testRangeSortWithOffsetLimit0(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, true) + util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(true)) util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC, b LIMIT 0 OFFSET 1") } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortTest.scala index 55401f3c0d503..d2f13cc93cb7a 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortTest.scala @@ -33,44 +33,44 @@ class SortTest extends TableTestBase { @Test def testNonRangeSortOnSingleFieldWithoutForceLimit(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, false) - util.tableEnv.getConfig.getConfiguration.setInteger( - ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, -1) + util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(false)) + util.tableEnv.getConfig.set( + ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, Integer.valueOf(-1)) util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC") } @Test def testNonRangeSortOnMultiFieldsWithoutForceLimit(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - TABLE_EXEC_RANGE_SORT_ENABLED, false) - util.tableEnv.getConfig.getConfiguration.setInteger( - ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, -1) + util.tableEnv.getConfig.set( + TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(false)) + util.tableEnv.getConfig.set( + ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, Integer.valueOf(-1)) util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC, b") } @Test def testNonRangeSortWithForceLimit(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - TABLE_EXEC_RANGE_SORT_ENABLED, false) - util.tableEnv.getConfig.getConfiguration.setInteger( - ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, 200) + util.tableEnv.getConfig.set( + TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(false)) + util.tableEnv.getConfig.set( + ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, Integer.valueOf(200)) util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC") } @Test def testRangeSortWithoutForceLimit(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.setInteger( - ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, -1) + util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig.set( + ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, Integer.valueOf(-1)) // exec node does not support range sort yet, so we verify rel plan here util.verifyRelPlan("SELECT * FROM MyTable ORDER BY a DESC") } @Test def testRangeSortWithForceLimit(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.setInteger( - ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, 200) + util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig.set( + ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, Integer.valueOf(200)) // exec node does not support range sort yet, so we verify rel plan here util.verifyRelPlan("SELECT * FROM MyTable ORDER BY a DESC") } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.scala index 33466b3723b2a..518019098e8ff 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.scala @@ -37,18 +37,18 @@ class SubplanReuseTest extends TableTestBase { @Before def before(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false)) util.addTableSource[(Int, Long, String)]("x", 'a, 'b, 'c) util.addTableSource[(Int, Long, String)]("y", 'd, 'e, 'f) } @Test def testDisableSubplanReuse(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(false)) val sqlQuery = """ |WITH r AS ( @@ -61,8 +61,8 @@ class SubplanReuseTest extends TableTestBase { @Test def testSubplanReuseWithDifferentRowType(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false)) // can not reuse because of different row-type val sqlQuery = """ @@ -75,8 +75,8 @@ class SubplanReuseTest extends TableTestBase { @Test def testEnableReuseTableSource(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(true)) val sqlQuery = """ |WITH t AS (SELECT x.a AS a, x.b AS b, y.d AS d, y.e AS e FROM x, y WHERE x.a = y.d) @@ -87,8 +87,8 @@ class SubplanReuseTest extends TableTestBase { @Test def testDisableReuseTableSource(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false)) val sqlQuery = """ |WITH t AS (SELECT * FROM x, y WHERE x.a = y.d) @@ -100,9 +100,9 @@ class SubplanReuseTest extends TableTestBase { @Test def testSubplanReuseOnSourceWithLimit(): Unit = { // TODO re-check this plan after PushLimitIntoTableSourceScanRule is introduced - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin") val sqlQuery = """ @@ -126,7 +126,7 @@ class SubplanReuseTest extends TableTestBase { @Test def testSubplanReuseOnCalc(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") val sqlQuery = """ @@ -166,7 +166,7 @@ class SubplanReuseTest extends TableTestBase { @Test def testSubplanReuseOnExchange(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") val sqlQuery = """ @@ -180,7 +180,7 @@ class SubplanReuseTest extends TableTestBase { @Test def testSubplanReuseOnHashAggregate(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, OperatorType.SortAgg.toString) val sqlQuery = """ @@ -192,7 +192,7 @@ class SubplanReuseTest extends TableTestBase { @Test def testSubplanReuseOnSortAggregate(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, OperatorType.HashAgg.toString) val sqlQuery = """ @@ -222,7 +222,7 @@ class SubplanReuseTest extends TableTestBase { @Test def testSubplanReuseOnSort(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, true) + util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(true)) val sqlQuery = """ |WITH r AS (SELECT c, SUM(a) a, SUM(b) b FROM x GROUP BY c ORDER BY a, b DESC) @@ -233,7 +233,7 @@ class SubplanReuseTest extends TableTestBase { @Test def testSubplanReuseOnLimit(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin") val sqlQuery = """ @@ -276,10 +276,10 @@ class SubplanReuseTest extends TableTestBase { @Test def testSubplanReuseOnSortMergeJoin(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin") - util.tableEnv.getConfig.getConfiguration.setBoolean( - BatchPhysicalSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true) + util.tableEnv.getConfig.set( + BatchPhysicalSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, Boolean.box(true)) val sqlQuery = """ |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%') @@ -290,7 +290,7 @@ class SubplanReuseTest extends TableTestBase { @Test def testSubplanReuseOnHashJoin(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") val sqlQuery = """ @@ -302,7 +302,7 @@ class SubplanReuseTest extends TableTestBase { @Test def testSubplanReuseOnNestedLoopJoin(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin") val sqlQuery = """ @@ -352,7 +352,7 @@ class SubplanReuseTest extends TableTestBase { @Test def testSubplanReuseOnCorrelate(): Unit = { util.addFunction("str_split", new StringSplit()) - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") val sqlQuery = """ @@ -391,7 +391,7 @@ class SubplanReuseTest extends TableTestBase { @Test def testNestedSubplanReuse(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg") val sqlQuery = """ @@ -427,7 +427,7 @@ class SubplanReuseTest extends TableTestBase { @Test def testBreakupDeadlockOnHashJoin(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") val sqlQuery = """ @@ -439,7 +439,7 @@ class SubplanReuseTest extends TableTestBase { @Test def testBreakupDeadlockOnNestedLoopJoin(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin") val sqlQuery = """ @@ -451,15 +451,15 @@ class SubplanReuseTest extends TableTestBase { @Test def testEnableReuseTableSourceOnNewSource(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(true)) testReuseOnNewSource() } @Test def testDisableReuseTableSourceOnNewSource(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false)) testReuseOnNewSource() } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSinkTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSinkTest.scala index 8373789b89cea..7a9d275ca719c 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSinkTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSinkTest.scala @@ -72,8 +72,9 @@ class TableSinkTest extends TableTestBase { |) |""".stripMargin) - util.tableEnv.getConfig.getConfiguration.setBoolean( - RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true) + util.tableEnv.getConfig.set( + RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, + Boolean.box(true)) val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS sum_a, c FROM MyTable GROUP BY c") util.tableEnv.createTemporaryView("table1", table1) val stmtSet = util.tableEnv.createStatementSet() diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.scala index 0edefde0230a5..cddc89da55866 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.scala @@ -18,7 +18,6 @@ package org.apache.flink.table.planner.plan.batch.sql -import org.apache.flink.table.api.config.TableConfigOptions import org.apache.flink.table.planner.plan.optimize.RelNodeBlockPlanBuilder import org.apache.flink.table.planner.utils._ @@ -220,8 +219,9 @@ class TableSourceTest extends TableTestBase { @Test def testTableHintWithDigestReuseForLogicalTableScan(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true) + util.tableEnv.getConfig.set( + RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, + Boolean.box(true)) util.tableEnv.executeSql( s""" |CREATE TABLE MySink ( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/GroupWindowTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/GroupWindowTest.scala index 7b82feeafad4a..35fef12836a34 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/GroupWindowTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/GroupWindowTest.scala @@ -39,7 +39,7 @@ class GroupWindowTest(aggStrategy: AggregatePhaseStrategy) extends TableTestBase @Before def before(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, aggStrategy.toString) util.addFunction("countFun", new CountAggFunction) util.addTableSource[(Int, Timestamp, Int, Long)]("MyTable", 'a, 'b, 'c, 'd) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.scala index c972b679638f3..1239ce00480cf 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.scala @@ -36,9 +36,9 @@ class HashAggregateTest(aggStrategy: AggregatePhaseStrategy) extends AggregateTe @Before def before(): Unit = { // disable sort agg - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, OperatorType.SortAgg.toString) - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, aggStrategy.toString) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.scala index e0dfe728b6c9e..af072fa2d491c 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.scala @@ -35,9 +35,9 @@ class SortAggregateTest(aggStrategy: AggregatePhaseStrategy) extends AggregateTe @Before def before(): Unit = { // disable hash agg - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, OperatorType.HashAgg.toString) - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, aggStrategy.toString) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.scala index 07d9b8582acbf..f738ec850e0c3 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.scala @@ -27,9 +27,10 @@ class BroadcastHashJoinTest extends JoinTestBase { @Before def before(): Unit = { - util.tableEnv.getConfig.getConfiguration.setLong( - OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.MaxValue) - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(Long.MaxValue)) + Long.box(10) + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin, NestedLoopJoin, ShuffleHashJoin") } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala index 6678332503a92..5b7730b2fd549 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala @@ -27,9 +27,9 @@ class BroadcastHashSemiAntiJoinTest extends SemiAntiJoinTestBase { @Before def before(): Unit = { - util.tableEnv.getConfig.getConfiguration.setLong( - OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.MaxValue) - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(Long.MaxValue)) + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin, NestedLoopJoin, ShuffleHashJoin") // the result plan may contains NestedLoopJoin (singleRowJoin) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.scala index 3dc3e7d46ebc2..70f0d9ee26401 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.scala @@ -188,7 +188,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase { |FROM ($sql2) AS T |GROUP BY b """.stripMargin - val programs = FlinkBatchProgram.buildProgram(testUtil.tableEnv.getConfig.getConfiguration) + val programs = FlinkBatchProgram.buildProgram(testUtil.tableEnv.getConfig) programs.remove(FlinkBatchProgram.PHYSICAL) testUtil.replaceBatchProgram(programs) testUtil.verifyRelPlan(sql) @@ -196,7 +196,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase { @Test def testLogicalPlanWithImplicitTypeCast(): Unit = { - val programs = FlinkBatchProgram.buildProgram(testUtil.tableEnv.getConfig.getConfiguration) + val programs = FlinkBatchProgram.buildProgram(testUtil.tableEnv.getConfig) programs.remove(FlinkBatchProgram.PHYSICAL) testUtil.replaceBatchProgram(programs) @@ -329,8 +329,8 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase { @Test def testReusing(): Unit = { - testUtil.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true) + testUtil.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true)) val sql1 = """ |SELECT b, a, sum(c) c, sum(d) d, PROCTIME() as proctime diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.scala index ff6b7249ffa4c..3b452886ba299 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.scala @@ -26,7 +26,7 @@ class NestedLoopJoinTest extends JoinTestBase { @Before def before(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin, HashJoin") } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala index c8548f8714200..b884b315c9f3a 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala @@ -26,7 +26,7 @@ class NestedLoopSemiAntiJoinTest extends SemiAntiJoinTestBase { @Before def before(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin, HashJoin") } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.scala index c262b516d5f5c..4b7966b5b004e 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.scala @@ -27,7 +27,7 @@ class ShuffledHashJoinTest extends JoinTestBase { @Before def before(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin, NestedLoopJoin, BroadcastHashJoin") } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala index f4814c4cdfb03..8315cfebb9494 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala @@ -27,7 +27,7 @@ class ShuffledHashSemiAntiJoinTest extends SemiAntiJoinTestBase { @Before def before(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin, NestedLoopJoin, BroadcastHashJoin") // the result plan may contains NestedLoopJoin (singleRowJoin) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.scala index b8b0a703214a5..2eacfc9515d91 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.scala @@ -27,7 +27,7 @@ class SortMergeJoinTest extends JoinTestBase { @Before def before(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin") } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala index e358c7bb39bdd..4a79b9886433e 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala @@ -27,7 +27,7 @@ class SortMergeSemiAntiJoinTest extends SemiAntiJoinTestBase { @Before def before(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin") // the result plan may contains NestedLoopJoin (singleRowJoin) // which is converted by BatchExecSingleRowJoinRule diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.scala index 14348aeed7a43..8ebaee682c54d 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.scala @@ -34,8 +34,6 @@ class CorrelateTest extends TableTestBase { val util = batchTestUtil() val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val func = new TableFunc1 - util.addFunction("func1", func) - val result1 = table.joinLateral(func('c) as 's).select('c, 's) util.verifyExecPlan(result1) @@ -46,8 +44,6 @@ class CorrelateTest extends TableTestBase { val util = batchTestUtil() val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val func = new TableFunc1 - util.addFunction("func1", func) - val result2 = table.joinLateral(func('c, "$") as 's).select('c, 's) util.verifyExecPlan(result2) } @@ -57,8 +53,6 @@ class CorrelateTest extends TableTestBase { val util = batchTestUtil() val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val func = new TableFunc1 - util.addFunction("func1", func) - val result = table.leftOuterJoinLateral(func('c) as 's).select('c, 's).where('s > "") util.verifyExecPlan(result) } @@ -68,8 +62,6 @@ class CorrelateTest extends TableTestBase { val util = batchTestUtil() val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val func = new TableFunc1 - util.addFunction("func1", func) - val result = table.leftOuterJoinLateral(func('c) as 's, true).select('c, 's) util.verifyExecPlan(result) } @@ -79,8 +71,6 @@ class CorrelateTest extends TableTestBase { val util = batchTestUtil() val sourceTable = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val func = new TableFunc0 - util.addFunction("func1", func) - val result = sourceTable.select('a, 'b, 'c) .joinLateral(func('c) as('d, 'e)) .select('c, 'd, 'e) @@ -106,8 +96,6 @@ class CorrelateTest extends TableTestBase { val sourceTable = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val func = new TableFunc0 - util.addFunction("func1", func) - val result = sourceTable.select('a, 'b, 'c) .joinLateral(func('c) as('d, 'e)) .select('c, 'd, 'e) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/stringexpr/CorrelateStringExpressionTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/stringexpr/CorrelateStringExpressionTest.scala index 95b340564fdf6..a4aa185d9aa27 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/stringexpr/CorrelateStringExpressionTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/stringexpr/CorrelateStringExpressionTest.scala @@ -29,9 +29,7 @@ class CorrelateStringExpressionTest extends TableTestBase { private val util = batchTestUtil() private val tab = util.addTableSource[(Int, Long, String)]("Table1", 'a, 'b, 'c) private val func1 = new TableFunc1 - util.addFunction("func1", func1) private val func2 = new TableFunc2 - util.addFunction("func2", func2) @Test def testCorrelateJoins1(): Unit = { @@ -61,7 +59,6 @@ class CorrelateStringExpressionTest extends TableTestBase { def testCorrelateJoins5(): Unit = { // test hierarchy generic type val hierarchy = new HierarchyTableFunction - util.addFunction("hierarchy", hierarchy) val scalaTable = tab.joinLateral( hierarchy('c) as('name, 'adult, 'len)).select('c, 'name, 'len, 'adult) util.verifyExecPlan(scalaTable) @@ -71,7 +68,6 @@ class CorrelateStringExpressionTest extends TableTestBase { def testCorrelateJoins6(): Unit = { // test pojo type val pojo = new PojoTableFunc - util.addFunction("pojo", pojo) val scalaTable = tab.joinLateral(pojo('c)).select('c, 'name, 'age) util.verifyExecPlan(scalaTable) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/CorrelateValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/CorrelateValidationTest.scala index d69383edfcdc0..611928ef5c663 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/CorrelateValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/CorrelateValidationTest.scala @@ -36,7 +36,6 @@ class CorrelateValidationTest extends TableTestBase { val util = batchTestUtil() val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val func = new TableFunc1 - util.addFunction("func1", func) val result = table .leftOuterJoinLateral(func('c) as 's, 'c === 's) .select('c, 's) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/JoinReorderTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/JoinReorderTestBase.scala index 0cf6659d8e464..2137811442dff 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/JoinReorderTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/JoinReorderTestBase.scala @@ -70,8 +70,8 @@ abstract class JoinReorderTestBase extends TableTestBase { "b5" -> new ColumnStats(200L, 0L, 8.0, 8, null, null) ))).build()) - util.getTableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_JOIN_REORDER_ENABLED, true) + util.getTableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_JOIN_REORDER_ENABLED, Boolean.box(true)) } @Test @@ -266,8 +266,8 @@ abstract class JoinReorderTestBase extends TableTestBase { "b8" -> builderB.build() ))).build()) - util.getTableEnv.getConfig.getConfiguration.setLong( - JoinDeriveNullFilterRule.TABLE_OPTIMIZER_JOIN_NULL_FILTER_THRESHOLD, 10000L) + util.getTableEnv.getConfig.set( + JoinDeriveNullFilterRule.TABLE_OPTIMIZER_JOIN_NULL_FILTER_THRESHOLD, Long.box(10000)) val sql = s""" |SELECT * FROM T6 diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/TableFactoryTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/TableFactoryTest.scala index c33a9d29e60e4..50fc5635ca3ec 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/TableFactoryTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/TableFactoryTest.scala @@ -47,7 +47,7 @@ class TableFactoryTest(isBatch: Boolean) extends TableTestBase { ObjectIdentifier.of("cat", "default", "t1"), ObjectIdentifier.of("cat", "default", "t2"), isBatch) - util.tableEnv.getConfig.getConfiguration.setBoolean(TestContextTableFactory.REQUIRED_KEY, true) + util.tableEnv.getConfig.set(TestContextTableFactory.REQUIRED_KEY, Boolean.box(true)) util.tableEnv.registerCatalog("cat", new GenericInMemoryCatalog("default") { override def getTableFactory: Optional[TableFactory] = Optional.of(factory) }) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/hint/OptionsHintTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/hint/OptionsHintTest.scala index 0ff55d13b0df6..ed7e224742792 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/hint/OptionsHintTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/hint/OptionsHintTest.scala @@ -75,9 +75,9 @@ class OptionsHintTest(param: Param) @Test def testOptionsWithGlobalConfDisabled(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( + util.tableEnv.getConfig.set( TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, - false) + Boolean.box(false)) expectedException.expect(isA(classOf[ValidationException])) expectedException.expectMessage(s"OPTIONS hint is allowed only when " + s"${TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED.key} is set to true") diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala index ef9db280baa26..b1a126518cf63 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala @@ -978,7 +978,7 @@ class FlinkRelMdHandlerTestBase { cluster, streamPhysicalTraits, streamTs, program, program.getOutputRowType) val streamExchange = new StreamPhysicalExchange( cluster, streamPhysicalTraits.replace(hash01), streamCalc, hash01) - val emitStrategy = WindowEmitStrategy(tableConfig.getConfiguration, tumblingGroupWindow) + val emitStrategy = WindowEmitStrategy(tableConfig, tumblingGroupWindow) val streamWindowAgg = new StreamPhysicalGroupWindowTableAggregate( cluster, streamPhysicalTraits, @@ -1620,7 +1620,7 @@ class FlinkRelMdHandlerTestBase { cluster, streamPhysicalTraits, streamTs, program, program.getOutputRowType) val streamExchange = new StreamPhysicalExchange( cluster, streamPhysicalTraits.replace(hash01), streamCalc, hash01) - val emitStrategy = WindowEmitStrategy(tableConfig.getConfiguration, tumblingGroupWindow) + val emitStrategy = WindowEmitStrategy(tableConfig, tumblingGroupWindow) val streamWindowAgg = new StreamPhysicalGroupWindowAggregate( cluster, streamPhysicalTraits, @@ -1759,7 +1759,7 @@ class FlinkRelMdHandlerTestBase { cluster, streamPhysicalTraits, streamTs, program, program.getOutputRowType) val streamExchange = new StreamPhysicalExchange( cluster, streamPhysicalTraits.replace(hash1), streamCalc, hash1) - val emitStrategy = WindowEmitStrategy(tableConfig.getConfiguration, tumblingGroupWindow) + val emitStrategy = WindowEmitStrategy(tableConfig, tumblingGroupWindow) val streamWindowAgg = new StreamPhysicalGroupWindowAggregate( cluster, streamPhysicalTraits, diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala index b459371ff7970..851397305c0bc 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala @@ -52,8 +52,8 @@ class JoinDeriveNullFilterRuleTest extends TableTestBase { .build() ) - util.tableEnv.getConfig.getConfiguration.setLong( - JoinDeriveNullFilterRule.TABLE_OPTIMIZER_JOIN_NULL_FILTER_THRESHOLD, 2000000) + util.tableEnv.getConfig.set( + JoinDeriveNullFilterRule.TABLE_OPTIMIZER_JOIN_NULL_FILTER_THRESHOLD, Long.box(2000000)) util.addTableSource("MyTable1", Array[TypeInformation[_]](Types.INT, Types.LONG, Types.STRING, Types.INT, Types.LONG), Array("a1", "b1", "c1", "d1", "e1"), diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRuleTest.scala index d809dc47de1aa..26d4324cbe5b2 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRuleTest.scala @@ -33,8 +33,8 @@ class SplitAggregateRuleTest extends TableTestBase { private val util = streamTestUtil() util.addTableSource[(Long, Int, String)]("MyTable", 'a, 'b, 'c) util.buildStreamProgram(FlinkStreamProgram.PHYSICAL) - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true)) @Test def testSingleDistinctAgg(): Unit = { @@ -163,16 +163,16 @@ class SplitAggregateRuleTest extends TableTestBase { @Test def testBucketsConfiguration(): Unit = { - util.tableEnv.getConfig.getConfiguration.setInteger( - OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_BUCKET_NUM, 100) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_BUCKET_NUM, Integer.valueOf(100)) val sqlQuery = "SELECT COUNT(DISTINCT c) FROM MyTable" util.verifyRelPlan(sqlQuery) } @Test def testMultipleDistinctAggOnSameColumn(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true)) val sqlQuery = s""" |SELECT @@ -189,8 +189,8 @@ class SplitAggregateRuleTest extends TableTestBase { @Test def testAggFilterClauseBothWithAvgAndCount(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true)) val sqlQuery = s""" |SELECT diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalHashAggRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalHashAggRuleTest.scala index c0c11281c8c2b..bfbae11a8d68d 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalHashAggRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalHashAggRuleTest.scala @@ -38,7 +38,7 @@ class EnforceLocalHashAggRuleTest extends EnforceLocalAggRuleTestBase { @Before override def setup(): Unit = { super.setup() - val program = FlinkBatchProgram.buildProgram(util.tableEnv.getConfig.getConfiguration) + val program = FlinkBatchProgram.buildProgram(util.tableEnv.getConfig) // remove the original BatchExecHashAggRule and add BatchExecHashAggRuleForOnePhase // to let the physical phase generate one phase aggregate program.getFlinkRuleSetProgram(FlinkBatchProgram.PHYSICAL) @@ -51,9 +51,9 @@ class EnforceLocalHashAggRuleTest extends EnforceLocalAggRuleTestBase { .replaceBatchProgram(program).build() util.tableEnv.getConfig.setPlannerConfig(calciteConfig) // only enabled HashAgg - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg") - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE") } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalSortAggRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalSortAggRuleTest.scala index e0eeca27e17eb..03a72a148ad6c 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalSortAggRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalSortAggRuleTest.scala @@ -42,7 +42,7 @@ class EnforceLocalSortAggRuleTest extends EnforceLocalAggRuleTestBase { super.setup() util.addFunction("weightedAvg", new WeightedAvg) - val program = FlinkBatchProgram.buildProgram(util.tableEnv.getConfig.getConfiguration) + val program = FlinkBatchProgram.buildProgram(util.tableEnv.getConfig) // remove the original BatchExecSortAggRule and add BatchExecSortAggRuleForOnePhase // to let the physical phase generate one phase aggregate program.getFlinkRuleSetProgram(FlinkBatchProgram.PHYSICAL) @@ -55,9 +55,9 @@ class EnforceLocalSortAggRuleTest extends EnforceLocalAggRuleTestBase { .replaceBatchProgram(program).build() util.tableEnv.getConfig.setPlannerConfig(calciteConfig) // only enabled SortAgg - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg") - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE") } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala index 1060fa80c37bf..fba9798ab7739 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala @@ -40,11 +40,11 @@ class RemoveRedundantLocalHashAggRuleTest extends TableTestBase { @Test def testRemoveRedundantLocalHashAgg_ShuffleKeyFromJoin(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg") // disable BroadcastHashJoin - util.tableEnv.getConfig.getConfiguration.setLong( - OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1)) val sqlQuery = """ |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%') @@ -55,10 +55,10 @@ class RemoveRedundantLocalHashAggRuleTest extends TableTestBase { @Test def testRemoveRedundantLocalHashAgg_ShuffleKeyFromRank(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg") - util.tableEnv.getConfig.getConfiguration.setBoolean( - BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, true) + util.tableEnv.getConfig.set( + BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, Boolean.box(true)) val sqlQuery = """ |SELECT a, SUM(b) FROM ( @@ -72,10 +72,10 @@ class RemoveRedundantLocalHashAggRuleTest extends TableTestBase { @Test def testUsingLocalAggCallFilters(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg") - util.tableEnv.getConfig.getConfiguration.setBoolean( - BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, true) + util.tableEnv.getConfig.set( + BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, Boolean.box(true)) val sqlQuery = "SELECT d, MAX(e), MAX(e) FILTER (WHERE a < 10), COUNT(DISTINCT c),\n" + "COUNT(DISTINCT c) FILTER (WHERE a > 5), COUNT(DISTINCT b) FILTER (WHERE b > 3)\n" + "FROM z GROUP BY d" diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ChangelogModeInferenceTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ChangelogModeInferenceTest.scala index 15f429392e629..a7fbbee95fb0d 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ChangelogModeInferenceTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ChangelogModeInferenceTest.scala @@ -19,7 +19,7 @@ package org.apache.flink.table.planner.plan.rules.physical.stream import org.apache.flink.api.common.time.Time -import org.apache.flink.table.api.{ExplainDetail, _} +import org.apache.flink.table.api.ExplainDetail import org.apache.flink.table.api.config.OptimizerConfigOptions import org.apache.flink.table.planner.plan.optimize.RelNodeBlockPlanBuilder import org.apache.flink.table.planner.plan.optimize.program.FlinkChangelogModeInferenceProgram @@ -162,7 +162,7 @@ class ChangelogModeInferenceTest extends TableTestBase { def testTwoLevelGroupByLocalGlobalOn(): Unit = { util.enableMiniBatch() util.tableEnv.getConfig.setIdleStateRetentionTime(Time.hours(1), Time.hours(2)) - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, AggregatePhaseStrategy.TWO_PHASE.toString) // two level unbounded groupBy @@ -223,9 +223,9 @@ class ChangelogModeInferenceTest extends TableTestBase { @Test def testPropagateUpdateKindAmongRelNodeBlocks(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( + util.tableEnv.getConfig.set( RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, - true) + Boolean.box(true)) util.addTable( """ |create table sink1 ( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ExpandWindowTableFunctionTransposeRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ExpandWindowTableFunctionTransposeRuleTest.scala index a3ec3d7a25f04..e1cdf315319e9 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ExpandWindowTableFunctionTransposeRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ExpandWindowTableFunctionTransposeRuleTest.scala @@ -83,8 +83,8 @@ class ExpandWindowTableFunctionTransposeRuleTest extends TableTestBase { |) |""".stripMargin) - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true)) } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.scala index 414b0447bf31e..1a39444aebefb 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.scala @@ -115,8 +115,8 @@ class DagOptimizationTest extends TableTestBase { @Test def testSingleSinkSplitOnUnion(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, false) + util.tableEnv.getConfig.set( + RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, Boolean.box(false)) val sqlQuery = "SELECT SUM(a) AS total_sum FROM " + "(SELECT a, c FROM MyTable UNION ALL SELECT d, f FROM MyTable1)" @@ -128,8 +128,9 @@ class DagOptimizationTest extends TableTestBase { @Test def testMultiSinks1(): Unit = { val stmtSet = util.tableEnv.createStatementSet() - util.tableEnv.getConfig.getConfiguration.setBoolean( - RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true) + util.tableEnv.getConfig.set( + RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, + Boolean.box(true)) val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS sum_a, c FROM MyTable GROUP BY c") util.tableEnv.registerTable("table1", table1) val table2 = util.tableEnv.sqlQuery("SELECT SUM(sum_a) AS total_sum FROM table1") @@ -151,8 +152,8 @@ class DagOptimizationTest extends TableTestBase { @Test def testMultiSinks2(): Unit = { val stmtSet = util.tableEnv.createStatementSet() - util.tableEnv.getConfig.getConfiguration.setBoolean( - RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, true) + util.tableEnv.getConfig.set( + RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, Boolean.box(true)) util.addTableSource[(Int, Long, String, Double, Boolean)]("MyTable2", 'a, 'b, 'c, 'd, 'e) val table1 = util.tableEnv.sqlQuery("SELECT a as a1, b as b1 FROM MyTable WHERE a <= 10") @@ -177,8 +178,8 @@ class DagOptimizationTest extends TableTestBase { @Test def testMultiSinks3(): Unit = { val stmtSet = util.tableEnv.createStatementSet() - util.tableEnv.getConfig.getConfiguration.setBoolean( - RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, true) + util.tableEnv.getConfig.set( + RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, Boolean.box(true)) util.addTableSource[(Int, Long, String, Double, Boolean)]("MyTable2", 'a, 'b, 'c, 'd, 'e) val table1 = util.tableEnv.sqlQuery("SELECT a AS a1, b AS b1 FROM MyTable WHERE a <= 10") @@ -231,8 +232,9 @@ class DagOptimizationTest extends TableTestBase { @Test def testMultiSinks5(): Unit = { val stmtSet = util.tableEnv.createStatementSet() - util.tableEnv.getConfig.getConfiguration.setBoolean( - RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true) + util.tableEnv.getConfig.set( + RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, + Boolean.box(true)) // test with non-deterministic udf util.tableEnv.registerFunction("random_udf", new NonDeterministicUdf()) val table1 = util.tableEnv.sqlQuery("SELECT random_udf(a) AS a, c FROM MyTable") @@ -256,8 +258,8 @@ class DagOptimizationTest extends TableTestBase { @Test def testMultiSinksWithUDTF(): Unit = { val stmtSet = util.tableEnv.createStatementSet() - util.tableEnv.getConfig.getConfiguration.setBoolean( - RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, false) + util.tableEnv.getConfig.set( + RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, Boolean.box(false)) util.addFunction("split", new TableFunc1) val sqlQuery1 = """ @@ -300,8 +302,8 @@ class DagOptimizationTest extends TableTestBase { @Test def testMultiSinksSplitOnUnion1(): Unit = { val stmtSet = util.tableEnv.createStatementSet() - util.tableEnv.getConfig.getConfiguration.setBoolean( - RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, false) + util.tableEnv.getConfig.set( + RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, Boolean.box(false)) val table = util.tableEnv.sqlQuery( "SELECT a, c FROM MyTable UNION ALL SELECT d, f FROM MyTable1") @@ -325,10 +327,11 @@ class DagOptimizationTest extends TableTestBase { @Test def testMultiSinksSplitOnUnion2(): Unit = { val stmtSet = util.tableEnv.createStatementSet() - util.tableEnv.getConfig.getConfiguration.setBoolean( - RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.setBoolean( - RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, false) + util.tableEnv.getConfig.set( + RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, + Boolean.box(true)) + util.tableEnv.getConfig.set( + RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, Boolean.box(false)) util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c) val sqlQuery1 = @@ -367,8 +370,8 @@ class DagOptimizationTest extends TableTestBase { @Test def testMultiSinksSplitOnUnion3(): Unit = { val stmtSet = util.tableEnv.createStatementSet() - util.tableEnv.getConfig.getConfiguration.setBoolean( - RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, false) + util.tableEnv.getConfig.set( + RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, Boolean.box(false)) util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c) val sqlQuery1 = "SELECT a, c FROM MyTable UNION ALL SELECT d, f FROM MyTable1" @@ -402,8 +405,8 @@ class DagOptimizationTest extends TableTestBase { @Test def testMultiSinksSplitOnUnion4(): Unit = { val stmtSet = util.tableEnv.createStatementSet() - util.tableEnv.getConfig.getConfiguration.setBoolean( - RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, false) + util.tableEnv.getConfig.set( + RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, Boolean.box(false)) util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c) val sqlQuery = @@ -509,8 +512,8 @@ class DagOptimizationTest extends TableTestBase { @Test def testMultiLevelViews(): Unit = { val stmtSet = util.tableEnv.createStatementSet() - util.tableEnv.getConfig.getConfiguration.setBoolean( - RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, false) + util.tableEnv.getConfig.set( + RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, Boolean.box(false)) val table1 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE c LIKE '%hello%'") util.tableEnv.registerTable("TempTable1", table1) @@ -552,8 +555,8 @@ class DagOptimizationTest extends TableTestBase { @Test def testSharedUnionNode(): Unit = { val stmtSet = util.tableEnv.createStatementSet() - util.tableEnv.getConfig.getConfiguration.setBoolean( - RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, false) + util.tableEnv.getConfig.set( + RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, Boolean.box(false)) val table1 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE c LIKE '%hello%'") util.tableEnv.registerTable("TempTable1", table1) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DeduplicateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DeduplicateTest.scala index a6ec4e894f10a..027dfac18f492 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DeduplicateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DeduplicateTest.scala @@ -70,7 +70,7 @@ class DeduplicateTest extends TableTestBase { @Test def testLastRowWithWindowOnRowtime(): Unit = { - util.tableEnv.getConfig.getConfiguration + util.tableEnv.getConfig .set(TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofMillis(500)) util.addTable( """ @@ -125,9 +125,9 @@ class DeduplicateTest extends TableTestBase { @Test def testMiniBatchInferFirstRowOnRowtime(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_MINIBATCH_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.setLong(TABLE_EXEC_MINIBATCH_SIZE, 3L) - util.tableEnv.getConfig.getConfiguration.set( + util.tableEnv.getConfig.set(TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig.set(TABLE_EXEC_MINIBATCH_SIZE, Long.box(3L)) + util.tableEnv.getConfig.set( TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)) val ddl = s""" @@ -175,9 +175,9 @@ class DeduplicateTest extends TableTestBase { @Test def testMiniBatchInferLastRowOnRowtime(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_MINIBATCH_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.setLong(TABLE_EXEC_MINIBATCH_SIZE, 3L) - util.tableEnv.getConfig.getConfiguration.set( + util.tableEnv.getConfig.set(TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig.set(TABLE_EXEC_MINIBATCH_SIZE, Long.box(3L)) + util.tableEnv.getConfig.set( TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)) val ddl = s""" diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.scala index 0cf621c6fa215..988ac69f407f6 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.scala @@ -72,13 +72,13 @@ class MiniBatchIntervalInferTest extends TableTestBase { |""".stripMargin) // enable mini-batch - util.tableEnv.getConfig.getConfiguration.setBoolean( - ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true) + util.tableEnv.getConfig.set( + ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(true)) } @Test def testMiniBatchOnly(): Unit = { - util.tableEnv.getConfig.getConfiguration + util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)) val sql = "SELECT b, COUNT(DISTINCT a), MAX(b), SUM(c) FROM MyTable1 GROUP BY b" util.verifyExecPlan(sql) @@ -86,7 +86,7 @@ class MiniBatchIntervalInferTest extends TableTestBase { @Test def testMiniBatchOnlyForDataStream(): Unit = { - util.tableEnv.getConfig.getConfiguration + util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)) val sql = "SELECT b, COUNT(DISTINCT a), MAX(b), SUM(c) FROM MyDataStream1 GROUP BY b" util.verifyExecPlan(sql) @@ -94,7 +94,7 @@ class MiniBatchIntervalInferTest extends TableTestBase { @Test def testRedundantWatermarkDefinition(): Unit = { - util.tableEnv.getConfig.getConfiguration + util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)) val sql = "SELECT b, COUNT(DISTINCT a), MAX(b), SUM(c) FROM wmTable1 GROUP BY b" util.verifyExecPlan(sql) @@ -103,7 +103,7 @@ class MiniBatchIntervalInferTest extends TableTestBase { @Test def testWindowWithEarlyFire(): Unit = { val tableConfig = util.tableEnv.getConfig - tableConfig.getConfiguration + tableConfig .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)) withEarlyFireDelay(tableConfig, Time.milliseconds(500)) val sql = @@ -124,7 +124,7 @@ class MiniBatchIntervalInferTest extends TableTestBase { @Test def testWindowCascade(): Unit = { - util.tableEnv.getConfig.getConfiguration + util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(3)) val sql = """ @@ -144,7 +144,7 @@ class MiniBatchIntervalInferTest extends TableTestBase { @Test def testIntervalJoinWithMiniBatch(): Unit = { - util.tableEnv.getConfig.getConfiguration.set( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)) val sql = @@ -165,7 +165,7 @@ class MiniBatchIntervalInferTest extends TableTestBase { @Test def testRowtimeRowsOverWithMiniBatch(): Unit = { - util.tableEnv.getConfig.getConfiguration.set( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)) val sql = @@ -187,7 +187,7 @@ class MiniBatchIntervalInferTest extends TableTestBase { util.addTableWithWatermark("Orders", util.tableEnv.from("MyDataStream1"), "rowtime", 0) util.addTableWithWatermark("RatesHistory", util.tableEnv.from("MyDataStream2"), "rowtime", 0) - util.tableEnv.getConfig.getConfiguration.set( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)) util.addTemporarySystemFunction( @@ -212,7 +212,7 @@ class MiniBatchIntervalInferTest extends TableTestBase { @Test def testMultiOperatorNeedsWatermark1(): Unit = { // infer result: miniBatchInterval=[Rowtime, 0ms] - util.tableEnv.getConfig.getConfiguration.set( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)) val sql = @@ -236,7 +236,7 @@ class MiniBatchIntervalInferTest extends TableTestBase { @Test def testMultiOperatorNeedsWatermark2(): Unit = { - util.tableEnv.getConfig.getConfiguration.set( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(6)) val sql = @@ -271,7 +271,7 @@ class MiniBatchIntervalInferTest extends TableTestBase { @Test def testMultiOperatorNeedsWatermark3(): Unit = { - util.tableEnv.getConfig.getConfiguration.set( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(6)) val sql = @@ -326,10 +326,9 @@ class MiniBatchIntervalInferTest extends TableTestBase { |) |""".stripMargin) - util.tableEnv.getConfig.getConfiguration.set( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofMillis(500)) - util.tableEnv.getConfig.getConfiguration.setLong( - ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, 300L) + util.tableEnv.getConfig.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, Long.box(300L)) val table1 = util.tableEnv.sqlQuery( """ @@ -393,7 +392,7 @@ class MiniBatchIntervalInferTest extends TableTestBase { @Test def testMiniBatchOnDataStreamWithRowTime(): Unit = { util.addDataStream[(Long, Int, String)]("T1", 'long, 'int, 'str, 'rowtime.rowtime) - util.tableEnv.getConfig.getConfiguration + util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)) val sql = """ @@ -409,7 +408,7 @@ class MiniBatchIntervalInferTest extends TableTestBase { @Test def testOverWindowMiniBatchOnDataStreamWithRowTime(): Unit = { util.addDataStream[(Long, Int, String)]("T1", 'long, 'int, 'str, 'rowtime.rowtime) - util.tableEnv.getConfig.getConfiguration + util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(3)) val sql = """ @@ -428,7 +427,7 @@ class MiniBatchIntervalInferTest extends TableTestBase { private def withEarlyFireDelay(tableConfig: TableConfig, interval: Time): Unit = { val intervalInMillis = interval.toMilliseconds - val earlyFireDelay: Duration = tableConfig.getConfiguration + val earlyFireDelay: Duration = tableConfig .getOptional(TABLE_EXEC_EMIT_EARLY_FIRE_DELAY) .orElse(null) if (earlyFireDelay != null && (earlyFireDelay.toMillis != intervalInMillis)) { // @@ -436,8 +435,7 @@ class MiniBatchIntervalInferTest extends TableTestBase { throw new RuntimeException("Currently not support different earlyFireInterval configs in " + "one job") } - tableConfig.getConfiguration.setBoolean(TABLE_EXEC_EMIT_EARLY_FIRE_ENABLED, Boolean.box(true)) - tableConfig.getConfiguration.set( - TABLE_EXEC_EMIT_EARLY_FIRE_DELAY, Duration.ofMillis(intervalInMillis)) + tableConfig.set(TABLE_EXEC_EMIT_EARLY_FIRE_ENABLED, Boolean.box(true)) + tableConfig.set(TABLE_EXEC_EMIT_EARLY_FIRE_DELAY, Duration.ofMillis(intervalInMillis)) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ModifiedMonotonicityTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ModifiedMonotonicityTest.scala index 4cb7b66137a96..c4b14ac5f9b97 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ModifiedMonotonicityTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ModifiedMonotonicityTest.scala @@ -67,9 +67,9 @@ class ModifiedMonotonicityTest extends TableTestBase { @Test def testMaxWithRetractOptimizeWithLocalGlobal(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true) - util.tableEnv.getConfig.getConfiguration + util.tableEnv.getConfig.set( + ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofMillis(100)) val query = "SELECT a1, max(a3) from (SELECT a1, a2, max(a3) as a3 FROM A GROUP BY a1, a2) " + "group by a1" @@ -78,9 +78,9 @@ class ModifiedMonotonicityTest extends TableTestBase { @Test def testMinWithRetractOptimizeWithLocalGlobal(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true) - util.tableEnv.getConfig.getConfiguration + util.tableEnv.getConfig.set( + ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofMillis(100)) val query = "SELECT min(a3) from (SELECT a1, a2, min(a3) as a3 FROM A GROUP BY a1, a2)" util.verifyRelPlan(query, ExplainDetail.CHANGELOG_MODE) @@ -88,9 +88,9 @@ class ModifiedMonotonicityTest extends TableTestBase { @Test def testMinCanNotOptimizeWithLocalGlobal(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true) - util.tableEnv.getConfig.getConfiguration + util.tableEnv.getConfig.set( + ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofMillis(100)) val query = "SELECT a1, MIN(a3) FROM (SELECT a1, a2, MAX(a3) AS a3 FROM A GROUP BY a1, a2) t GROUP BY a1" diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RankTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RankTest.scala index 3963652956b8f..18d292f7712d6 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RankTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RankTest.scala @@ -840,8 +840,9 @@ class RankTest extends TableTestBase { @Test def testUpdatableRankAfterIntermediateScan(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true) + util.tableEnv.getConfig.set( + RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, + Boolean.box(true)) util.tableEnv.executeSql( """ |CREATE VIEW v1 AS diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SubplanReuseTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SubplanReuseTest.scala index 12a851ee6d274..c6b092225e9bc 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SubplanReuseTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SubplanReuseTest.scala @@ -34,18 +34,18 @@ class SubplanReuseTest extends TableTestBase { @Before def before(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false)) util.addTableSource[(Int, Long, String)]("x", 'a, 'b, 'c) util.addTableSource[(Int, Long, String)]("y", 'd, 'e, 'f) } @Test def testDisableSubplanReuse(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(false)) val sqlQuery = """ |WITH r AS ( @@ -58,8 +58,8 @@ class SubplanReuseTest extends TableTestBase { @Test def testSubplanReuseWithDifferentRowType(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false)) // can not reuse because of different row-type val sqlQuery = """ @@ -72,8 +72,8 @@ class SubplanReuseTest extends TableTestBase { @Test def testEnableReuseTableSource(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(true)) val sqlQuery = """ |WITH t AS (SELECT x.a AS a, x.b AS b, y.d AS d, y.e AS e FROM x, y WHERE x.a = y.d) @@ -84,8 +84,8 @@ class SubplanReuseTest extends TableTestBase { @Test def testDisableReuseTableSource(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false)) val sqlQuery = """ |WITH t AS (SELECT * FROM x, y WHERE x.a = y.d) @@ -184,7 +184,7 @@ class SubplanReuseTest extends TableTestBase { @Test def testSubplanReuseOnLimit(): Unit = { - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin") val sqlQuery = """ @@ -303,15 +303,15 @@ class SubplanReuseTest extends TableTestBase { @Test def testEnableReuseTableSourceOnNewSource(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(true)) testReuseOnNewSource() } @Test def testDisableReuseTableSourceOnNewSource(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false)) testReuseOnNewSource() } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala index 598a2ddb980e6..f4a4c72033f96 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala @@ -19,11 +19,15 @@ package org.apache.flink.table.planner.plan.stream.sql import org.apache.flink.api.scala._ +import org.apache.flink.core.testutils.FlinkAssertions +import org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches import org.apache.flink.table.api._ import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.planner.expressions.utils.Func0 import org.apache.flink.table.planner.factories.TestValuesTableFactory.MockedLookupTableSource import org.apache.flink.table.planner.utils.TableTestBase + +import org.assertj.core.api.Assertions.assertThatThrownBy import org.junit.Test class TableScanTest extends TableTestBase { @@ -99,6 +103,25 @@ class TableScanTest extends TableTestBase { util.verifyExecPlan("SELECT * FROM MetadataTable") } + @Test + def testDDLWithMetadataThatConflictsWithPhysicalColumn(): Unit = { + util.addTable( + s""" + |CREATE TABLE MetadataTable ( + | `timestamp` TIMESTAMP(9), + | `metadata_timestamp` TIMESTAMP(0) METADATA FROM 'timestamp', + | `other` STRING METADATA, + | `computed_other` AS UPPER(`other`), + | `computed_timestamp` AS CAST(`metadata_timestamp` AS STRING) + |) WITH ( + | 'connector' = 'values', + | 'bounded' = 'false', + | 'readable-metadata' = 'timestamp:TIMESTAMP(0), other:STRING' + |) + """.stripMargin) + util.verifyExecPlan("SELECT * FROM MetadataTable") + } + @Test def testDDLWithMetadataColumnProjectionPushDown(): Unit = { // tests reordering, skipping, and casting of metadata @@ -158,6 +181,54 @@ class TableScanTest extends TableTestBase { util.verifyExecPlan("SELECT * FROM src WHERE a > 1") } + @Test + def testDDLWithMultipleColumnsFromSameMetadataKey(): Unit = { + assertThatThrownBy(() => + util.tableEnv.executeSql( + """ + |CREATE TABLE source ( + | a INT METADATA, + | b INT METADATA FROM 'a' + |) WITH ( + | 'connector' = 'COLLECTION' + |) + |""".stripMargin)).satisfies( + FlinkAssertions.anyCauseMatches( + classOf[ValidationException], + "The column `a` and `b` in the table are both from the same metadata key 'a'. " + + "Please specify one of the columns as the metadata column and use the computed column" + + " syntax to specify the others.")) + } + + @Test + def testDDLWithMultipleColumnsFromSameMetadataKey2(): Unit = { + util.tableEnv.executeSql( + """ + |CREATE TABLE source ( + | a INT METADATA + |) WITH ( + | 'connector' = 'COLLECTION' + |) + |""".stripMargin) + assertThatThrownBy(() => + util.tableEnv.executeSql( + """ + |CREATE TABLE like_source ( + | b INT METADATA FROM 'a' + |) + |WITH ( + | 'connector' = 'COLLECTION' + |) LIKE source ( + | INCLUDING METADATA + |) + |""".stripMargin + )).satisfies(anyCauseMatches( + "The column `a` and `b` in the table are both from the same metadata key 'a'. " + + "Please specify one of the columns as the metadata column and use the computed column" + + " syntax to specify the others." + )) + } + @Test def testKeywordsWithWatermarkComputedColumn(): Unit = { // Create table with field as atom expression. @@ -288,8 +359,8 @@ class TableScanTest extends TableTestBase { @Test def testJoinOnChangelogSourceWithEventsDuplicate(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - ExecutionConfigOptions.TABLE_EXEC_SOURCE_CDC_EVENTS_DUPLICATE, true) + util.tableEnv.getConfig.set( + ExecutionConfigOptions.TABLE_EXEC_SOURCE_CDC_EVENTS_DUPLICATE, Boolean.box(true)) verifyJoinOnSource("I,UB,UA") } @@ -356,8 +427,8 @@ class TableScanTest extends TableTestBase { @Test def testChangelogSourceWithEventsDuplicate(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - ExecutionConfigOptions.TABLE_EXEC_SOURCE_CDC_EVENTS_DUPLICATE, true) + util.tableEnv.getConfig.set( + ExecutionConfigOptions.TABLE_EXEC_SOURCE_CDC_EVENTS_DUPLICATE, Boolean.box(true)) util.addTable( """ |CREATE TABLE src ( @@ -668,8 +739,8 @@ class TableScanTest extends TableTestBase { | 'changelog-mode' = 'I,UB,UA,D' |) """.stripMargin) - util.tableEnv.getConfig.getConfiguration.setBoolean( - ExecutionConfigOptions.TABLE_EXEC_SOURCE_CDC_EVENTS_DUPLICATE, true) + util.tableEnv.getConfig.set( + ExecutionConfigOptions.TABLE_EXEC_SOURCE_CDC_EVENTS_DUPLICATE, Boolean.box(true)) thrown.expect(classOf[TableException]) thrown.expectMessage("Configuration 'table.exec.source.cdc-events-duplicate' is enabled " + diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.scala index d5ef3f380f986..f568041691f96 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.scala @@ -429,6 +429,35 @@ class TableSinkTest extends TableTestBase { util.verifyRelPlan(stmtSet) } + @Test + def testMetadataColumnThatConflictsWithPhysicalColumn(): Unit = { + util.addTable( + s""" + |CREATE TABLE MetadataTable ( + | `metadata_1` DOUBLE, + | `m_1` STRING METADATA FROM 'metadata_1' VIRTUAL, + | `m_2` BIGINT METADATA FROM 'metadata_2', + | `metadata_2` DOUBLE, + | `other` STRING + |) WITH ( + | 'connector' = 'values', + | 'readable-metadata' = 'metadata_1:STRING, metadata_2:BIGINT', + | 'writable-metadata' = 'metadata_1:STRING, metadata_2:BIGINT' + |) + """.stripMargin) + + val sql = + """ + |INSERT INTO MetadataTable + |SELECT `metadata_1`, `m_2`, `metadata_2`, `other` + |FROM MetadataTable + |""".stripMargin + val stmtSet = util.tableEnv.createStatementSet() + stmtSet.addInsertSql(sql) + + util.verifyRelPlan(stmtSet) + } + @Test def testSinkDisorderChangeLogWithJoin(): Unit = { util.tableEnv.executeSql( @@ -504,7 +533,7 @@ class TableSinkTest extends TableTestBase { @Test def testAppendStreamToSinkWithPkNoKeyBy(): Unit = { val tEnv = util.tableEnv - tEnv.getConfig.getConfiguration.set(ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE, + tEnv.getConfig.set(ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE, ExecutionConfigOptions.SinkKeyedShuffle.NONE) tEnv.executeSql( """ @@ -535,7 +564,8 @@ class TableSinkTest extends TableTestBase { @Test def testAppendStreamToSinkWithPkForceKeyBy(): Unit = { util.getStreamEnv.setParallelism(4) val tEnv = util.tableEnv - tEnv.getConfig.getConfiguration.set(ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE, + tEnv.getConfig.set( + ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE, ExecutionConfigOptions.SinkKeyedShuffle.FORCE) tEnv.executeSql( """ @@ -566,7 +596,8 @@ class TableSinkTest extends TableTestBase { @Test def testSingleParallelismAppendStreamToSinkWithPkForceKeyBy(): Unit = { util.getStreamEnv.setParallelism(1) val tEnv = util.tableEnv - tEnv.getConfig.getConfiguration.set(ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE, + tEnv.getConfig.set( + ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE, ExecutionConfigOptions.SinkKeyedShuffle.FORCE) tEnv.executeSql( """ @@ -597,7 +628,8 @@ class TableSinkTest extends TableTestBase { @Test def testAppendStreamToSinkWithoutPkForceKeyBy(): Unit = { util.getStreamEnv.setParallelism(4) val tEnv = util.tableEnv - tEnv.getConfig.getConfiguration.set(ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE, + tEnv.getConfig.set( + ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE, ExecutionConfigOptions.SinkKeyedShuffle.FORCE) tEnv.executeSql( """ @@ -627,7 +659,8 @@ class TableSinkTest extends TableTestBase { @Test def testAppendStreamToSinkWithoutPkForceKeyBySingleParallelism(): Unit = { util.getStreamEnv.setParallelism(4) val tEnv = util.tableEnv - tEnv.getConfig.getConfiguration.set(ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE, + tEnv.getConfig.set( + ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE, ExecutionConfigOptions.SinkKeyedShuffle.FORCE) tEnv.executeSql( """ @@ -656,7 +689,8 @@ class TableSinkTest extends TableTestBase { @Test def testChangelogStreamToSinkWithPkDifferentParallelism(): Unit = { util.getStreamEnv.setParallelism(1) val tEnv = util.tableEnv - tEnv.getConfig.getConfiguration.set(ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE, + tEnv.getConfig.set( + ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE, ExecutionConfigOptions.SinkKeyedShuffle.AUTO) tEnv.executeSql( """ @@ -685,10 +719,12 @@ class TableSinkTest extends TableTestBase { util.verifyExplain(stmtSet, ExplainDetail.JSON_EXECUTION_PLAN) } - @Test def testChangelogStreamToSinkWithPkSingleParallelism(): Unit = { + @Test + def testChangelogStreamToSinkWithPkSingleParallelism(): Unit = { util.getStreamEnv.setParallelism(4) val tEnv = util.tableEnv - tEnv.getConfig.getConfiguration.set(ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE, + tEnv.getConfig.set( + ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE, ExecutionConfigOptions.SinkKeyedShuffle.FORCE) tEnv.executeSql( """ @@ -764,6 +800,27 @@ class TableSinkTest extends TableTestBase { util.verifyAstPlan(stmtSet, ExplainDetail.CHANGELOG_MODE) } + + @Test + def testInsertPartColumn(): Unit = { + util.addTable( + s""" + |CREATE TABLE zm_test ( + | `a` BIGINT, + | `m1` MAP, + | `m2` MAP, + | `m3` MAP, + | `m4` MAP + |) WITH ( + | 'connector' = 'values', + | 'sink-insert-only' = 'true' + |) + |""".stripMargin) + val stmtSet = util.tableEnv.createStatementSet() + stmtSet.addInsertSql( + "INSERT INTO zm_test(`a`) SELECT `a` FROM MyTable") + util.verifyRelPlan(stmtSet, ExplainDetail.CHANGELOG_MODE) + } } /** tests table factory use ParallelSourceFunction which support parallelism by env*/ diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala index 883415d59d05a..48274d4633d7c 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala @@ -64,18 +64,18 @@ class AggregateTest extends TableTestBase { @Test def testAggWithMiniBatch(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.set( + util.tableEnv.getConfig.set( + ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)) util.verifyExecPlan("SELECT b, COUNT(DISTINCT a), MAX(b), SUM(c) FROM MyTable GROUP BY b") } @Test def testAggAfterUnionWithMiniBatch(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.set( + util.tableEnv.getConfig.set( + ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)) val query = """ @@ -97,9 +97,9 @@ class AggregateTest extends TableTestBase { @Test def testLocalGlobalAggAfterUnion(): Unit = { // enable local global optimize - util.tableEnv.getConfig.getConfiguration.setBoolean( - ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.set( + util.tableEnv.getConfig.set( + ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)) val sql = @@ -130,9 +130,9 @@ class AggregateTest extends TableTestBase { @Test def testAggWithFilterClauseWithLocalGlobal(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true) - util.tableEnv.getConfig.getConfiguration.set( + util.tableEnv.getConfig.set( + ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(true)) + util.tableEnv.getConfig.set( ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)) val sql = diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.scala index bc661d10b3321..795d26826a831 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.scala @@ -44,13 +44,14 @@ class DistinctAggregateTest( def before(): Unit = { util.tableEnv.getConfig.setIdleStateRetentionTime(Time.hours(1), Time.hours(2)) util.enableMiniBatch() - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, aggPhaseEnforcer.toString) - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, splitDistinctAggEnabled) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, + Boolean.box(splitDistinctAggEnabled)) // disable incremental agg - util.tableEnv.getConfig.getConfiguration.setBoolean( - IncrementalAggregateRule.TABLE_OPTIMIZER_INCREMENTAL_AGG_ENABLED, false) + util.tableEnv.getConfig.set( + IncrementalAggregateRule.TABLE_OPTIMIZER_INCREMENTAL_AGG_ENABLED, Boolean.box(false)) } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/GroupWindowTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/GroupWindowTest.scala index 90739d56246b5..d0a039f36b80f 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/GroupWindowTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/GroupWindowTest.scala @@ -433,8 +433,8 @@ class GroupWindowTest extends TableTestBase { @Test def testWindowAggregateWithLateFire(): Unit = { - util.conf.getConfiguration.setBoolean(TABLE_EXEC_EMIT_LATE_FIRE_ENABLED, true) - util.conf.getConfiguration.set(TABLE_EXEC_EMIT_LATE_FIRE_DELAY, Duration.ofSeconds(5)) + util.conf.set(TABLE_EXEC_EMIT_LATE_FIRE_ENABLED, Boolean.box(true)) + util.conf.set(TABLE_EXEC_EMIT_LATE_FIRE_DELAY, Duration.ofSeconds(5)) util.conf.setIdleStateRetentionTime(Time.hours(1), Time.hours(2)) val sql = """ @@ -447,9 +447,9 @@ class GroupWindowTest extends TableTestBase { @Test def testWindowAggregateWithAllowLateness(): Unit = { - util.conf.getConfiguration.setBoolean(TABLE_EXEC_EMIT_LATE_FIRE_ENABLED, true) - util.conf.getConfiguration.set(TABLE_EXEC_EMIT_LATE_FIRE_DELAY, Duration.ofSeconds(5)) - util.conf.getConfiguration.set(TABLE_EXEC_EMIT_ALLOW_LATENESS, Duration.ofHours(1)) + util.conf.set(TABLE_EXEC_EMIT_LATE_FIRE_ENABLED, Boolean.box(true)) + util.conf.set(TABLE_EXEC_EMIT_LATE_FIRE_DELAY, Duration.ofSeconds(5)) + util.conf.set(TABLE_EXEC_EMIT_ALLOW_LATENESS, Duration.ofHours(1)) val sql = """ |SELECT TUMBLE_START(`rowtime`, INTERVAL '1' SECOND), COUNT(*) cnt @@ -461,9 +461,9 @@ class GroupWindowTest extends TableTestBase { @Test def testWindowAggregateWithInvalidAllowLateness(): Unit = { - util.conf.getConfiguration.setBoolean(TABLE_EXEC_EMIT_LATE_FIRE_ENABLED, true) - util.conf.getConfiguration.set(TABLE_EXEC_EMIT_LATE_FIRE_DELAY, Duration.ofSeconds(5)) - util.conf.getConfiguration.set(TABLE_EXEC_EMIT_ALLOW_LATENESS, Duration.ofSeconds(1)) + util.conf.set(TABLE_EXEC_EMIT_LATE_FIRE_ENABLED, Boolean.box(true)) + util.conf.set(TABLE_EXEC_EMIT_LATE_FIRE_DELAY, Duration.ofSeconds(5)) + util.conf.set(TABLE_EXEC_EMIT_ALLOW_LATENESS, Duration.ofSeconds(1)) val sql = """ |SELECT TUMBLE_START(`rowtime`, INTERVAL '1' SECOND), COUNT(*) cnt diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.scala index 75ebee11d6f39..8863204b74ceb 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.scala @@ -37,8 +37,8 @@ class IncrementalAggregateTest( override def before(): Unit = { super.before() // enable incremental agg - util.tableEnv.getConfig.getConfiguration.setBoolean( - IncrementalAggregateRule.TABLE_OPTIMIZER_INCREMENTAL_AGG_ENABLED, true) + util.tableEnv.getConfig.set( + IncrementalAggregateRule.TABLE_OPTIMIZER_INCREMENTAL_AGG_ENABLED, Boolean.box(true)) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/TwoStageAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/TwoStageAggregateTest.scala index 93c157d5731d7..96757618e5252 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/TwoStageAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/TwoStageAggregateTest.scala @@ -35,7 +35,7 @@ class TwoStageAggregateTest extends TableTestBase { def before(): Unit = { util.enableMiniBatch() util.tableEnv.getConfig.setIdleStateRetentionTime(Time.hours(1), Time.hours(2)) - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, AggregatePhaseStrategy.TWO_PHASE.toString) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.scala index d1b9a6222a0be..5eddd7165ec9e 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.scala @@ -69,7 +69,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl |""".stripMargin) // set agg-phase strategy - util.tableEnv.getConfig.getConfiguration.setString( + util.tableEnv.getConfig.set( OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, aggPhaseEnforcer.toString) } @@ -284,8 +284,8 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl @Test def testTumble_DistinctSplitEnabled(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true)) val sql = """ |SELECT @@ -304,8 +304,8 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl @Test def testTumble_DistinctOnWindowColumns(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true)) // window_time is used in agg arg, thus we shouldn't merge WindowTVF into WindowAggregate. // actually, after expanded, there's HASH_CODE(window_time), // and thus we shouldn't transpose WindowTVF and Expand too. @@ -328,8 +328,8 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl def testTumble_DoNotSplitProcessingTimeWindow(): Unit = { assumeTrue(isTwoPhase) // the processing-time window aggregate with distinct shouldn't be split into two-level agg - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true)) val sql = """ |SELECT @@ -426,8 +426,8 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl @Test def testCumulate_DistinctSplitEnabled(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true)) val sql = """ |SELECT @@ -488,8 +488,8 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl @Test def testHop_DistinctSplitEnabled(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true)) val sql = """ |SELECT @@ -792,8 +792,8 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl @Test def testTumble_GroupingSetsDistinctSplitEnabled(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true)) val sql = """ |SELECT @@ -855,8 +855,8 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl @Test def testCantMergeWindowTVF_GroupingSetsDistinctOnWindowColumns(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true)) // window_time is used in agg arg, thus we shouldn't merge WindowTVF into WindowAggregate. // actually, after expanded, there's HASH_CODE(window_time), // and thus we shouldn't transpose WindowTVF and Expand too. @@ -891,8 +891,8 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl @Test def testHop_GroupingSets_DistinctSplitEnabled(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true)) val sql = """ |SELECT @@ -954,8 +954,8 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl @Test def testCumulate_GroupingSets_DistinctSplitEnabled(): Unit = { - util.tableEnv.getConfig.getConfiguration.setBoolean( - OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true) + util.tableEnv.getConfig.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true)) val sql = """ |SELECT diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.scala index 339e7cad4d4fd..ff1be68df1ced 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.scala @@ -36,8 +36,6 @@ class CorrelateTest extends TableTestBase { val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val function = new TableFunc1 - util.addFunction("func1", function) - val result1 = table.joinLateral(function('c) as 's).select('c, 's) util.verifyExecPlan(result1) } @@ -48,7 +46,6 @@ class CorrelateTest extends TableTestBase { val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val function = new TableFunc1 - util.addFunction("func1", function) // test overloading val result2 = table.joinLateral(function('c, "$") as 's).select('c, 's) util.verifyExecPlan(result2) @@ -59,8 +56,6 @@ class CorrelateTest extends TableTestBase { val util = streamTestUtil() val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val function = new TableFunc1 - util.addFunction("func1", function) - val result = table.leftOuterJoinLateral(function('c) as 's, true).select('c, 's) util.verifyExecPlan(result) } @@ -70,7 +65,6 @@ class CorrelateTest extends TableTestBase { val util = streamTestUtil() val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val function = new TableFunc2 - util.addFunction("func2", function) val scalarFunc = new Func13("pre") val result = table.joinLateral( @@ -84,8 +78,6 @@ class CorrelateTest extends TableTestBase { val util = streamTestUtil() val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val function = new HierarchyTableFunction - util.addFunction("hierarchy", function) - val result = table.joinLateral(function('c) as ('name, 'adult, 'len)) util.verifyExecPlan(result) } @@ -95,8 +87,6 @@ class CorrelateTest extends TableTestBase { val util = streamTestUtil() val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val function = new PojoTableFunc - util.addFunction("pojo", function) - val result = table.joinLateral(function('c)) util.verifyExecPlan(result) } @@ -106,8 +96,6 @@ class CorrelateTest extends TableTestBase { val util = streamTestUtil() val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val function = new TableFunc2 - util.addFunction("func2", function) - val result = table .joinLateral(function('c) as ('name, 'len)) .select('c, 'name, 'len) @@ -120,8 +108,6 @@ class CorrelateTest extends TableTestBase { val util = streamTestUtil() val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val function = new TableFunc1 - util.addFunction("func1", function) - val result = table.joinLateral(function('c.substring(2)) as 's) util.verifyExecPlan(result) } @@ -131,8 +117,6 @@ class CorrelateTest extends TableTestBase { val util = streamTestUtil() val sourceTable = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val function = new TableFunc0 - util.addFunction("func1", function) - val result = sourceTable.select('a, 'b, 'c) .joinLateral(function('c) as('d, 'e)) .select('c, 'd, 'e) @@ -158,7 +142,6 @@ class CorrelateTest extends TableTestBase { val sourceTable = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val function = new TableFunc0 - util.addFunction("func1", function) val result = sourceTable.select('a, 'b, 'c) .joinLateral(function('c) as('d, 'e)) .select('c, 'd, 'e) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CorrelateValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CorrelateValidationTest.scala index ec2401fd6b241..a991cc531283b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CorrelateValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CorrelateValidationTest.scala @@ -68,11 +68,10 @@ class CorrelateValidationTest extends TableTestBase { util.addFunction("func0", Func0) // SQL API call - // NOTE: it doesn't throw an exception but an AssertionError, maybe a Calcite bug expectExceptionThrown( util.tableEnv.sqlQuery("SELECT * FROM MyTable, LATERAL TABLE(func0(a))"), null, - classOf[AssertionError]) + classOf[ValidationException]) //========== throw exception when the parameters is not correct =============== // Java Table API call diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala index e591f1076fea0..45976cef02b20 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala @@ -616,12 +616,23 @@ class CalcITCase extends BatchTestBase { // )) // } + @Test + def testInNonConstantValue(): Unit = { + checkResult( + "SELECT a FROM Table3 WHERE a IN (CAST(b AS INT), 21)", + Seq(row(1), row(2), row(21))) + } + @Test def testInSmallValues(): Unit = { checkResult( "SELECT a FROM Table3 WHERE a in (1, 2)", Seq(row(1), row(2))) + checkResult( + "SELECT a FROM Table3 WHERE a in (1, 2, NULL)", + Seq(row(1), row(2))) + checkResult( "SELECT a FROM Table3 WHERE a in (1, 2) and b = 2", Seq(row(2))) @@ -1640,10 +1651,10 @@ class CalcITCase extends BatchTestBase { tEnv.executeSql(ddl) checkResult( - "select a from MyTable where cast(b as boolean)", + "select a from MyTable where try_cast(b as boolean)", Seq(row(1))) checkResult( - "select cast(b as boolean) from MyTable", + "select try_cast(b as boolean) from MyTable", Seq(row(true), row(false), row(null), row(null))) } @@ -1847,4 +1858,12 @@ class CalcITCase extends BatchTestBase { LocalTime.of(16, 50, 1, 123000000) ))) } + + @Test + def testTryCast(): Unit = { + checkResult("SELECT TRY_CAST('invalid' AS INT)", Seq(row(null))) + checkResult( + "SELECT TRY_CAST(g AS DOUBLE) FROM testTable", + Seq(row(null), row(null), row(null))) + } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala index c12523548acaf..1668880b6557f 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala @@ -43,7 +43,7 @@ class AggregateJoinTransposeITCase extends BatchTestBase { @Before override def before(): Unit = { super.before() - val programs = FlinkBatchProgram.buildProgram(tEnv.getConfig.getConfiguration) + val programs = FlinkBatchProgram.buildProgram(tEnv.getConfig) // remove FlinkAggregateJoinTransposeRule from logical program (volcano planner) programs.getFlinkRuleSetProgram(FlinkBatchProgram.LOGICAL) .getOrElse(throw new TableException(s"${FlinkBatchProgram.LOGICAL} does not exist")) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/CorrelateITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/CorrelateITCase.scala index 17a35c2b5778b..a125f93c717dd 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/CorrelateITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/CorrelateITCase.scala @@ -222,7 +222,8 @@ class CorrelateITCase extends BatchTestBase { 'a.cast(DataTypes.TINYINT) as 'a, 'a.cast(DataTypes.SMALLINT) as 'b, 'b.cast(DataTypes.FLOAT) as 'c) - .joinLateral(tFunc('a, 'b, 'c) as ('a2, 'b2, 'c2)) + .joinLateral( + tFunc('a.ifNull(0.toByte), 'b.ifNull(0.toShort), 'c.ifNull(0.toFloat)) as ('a2, 'b2, 'c2)) val results = executeQuery(result) val expected = Seq( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/GroupAggregateHarnessTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/GroupAggregateHarnessTest.scala index 39b73ef0cbd45..8a81fc662de3d 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/GroupAggregateHarnessTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/GroupAggregateHarnessTest.scala @@ -56,8 +56,7 @@ class GroupAggregateHarnessTest(mode: StateBackendMode, miniBatch: MiniBatchMode override def before(): Unit = { super.before() val setting = EnvironmentSettings.newInstance().inStreamingMode().build() - val config = new TestTableConfig - this.tEnv = StreamTableEnvironmentImpl.create(env, setting, config) + this.tEnv = StreamTableEnvironmentImpl.create(env, setting) // set mini batch val tableConfig = tEnv.getConfig miniBatch match { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/HarnessTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/HarnessTestBase.scala index 6aff43d554341..f634b1528e09b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/HarnessTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/HarnessTestBase.scala @@ -17,7 +17,6 @@ */ package org.apache.flink.table.planner.runtime.harness -import org.apache.flink.api.common.time.Time import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.dag.Transformation import org.apache.flink.api.java.functions.KeySelector @@ -30,14 +29,13 @@ import org.apache.flink.streaming.api.scala.DataStream import org.apache.flink.streaming.api.transformations.{OneInputTransformation, PartitionTransformation} import org.apache.flink.streaming.api.watermark.Watermark import org.apache.flink.streaming.util.{KeyedOneInputStreamOperatorTestHarness, OneInputStreamOperatorTestHarness} -import org.apache.flink.table.api.TableConfig import org.apache.flink.table.data.RowData import org.apache.flink.table.planner.JLong import org.apache.flink.table.planner.runtime.utils.StreamingTestBase import org.apache.flink.table.planner.runtime.utils.StreamingWithStateTestBase.{HEAP_BACKEND, ROCKSDB_BACKEND, StateBackendMode} + import org.junit.runners.Parameterized -import java.time.Duration import java.util import scala.collection.JavaConversions._ @@ -120,23 +118,6 @@ class HarnessTestBase(mode: StateBackendMode) extends StreamingTestBase { def dropWatermarks(elements: Array[AnyRef]): util.Collection[AnyRef] = { elements.filter(e => !e.isInstanceOf[Watermark]).toList } - - class TestTableConfig extends TableConfig { - - private var minIdleStateRetentionTime = 0L - - private var maxIdleStateRetentionTime = 0L - - override def getMinIdleStateRetentionTime: Long = minIdleStateRetentionTime - - override def getMaxIdleStateRetentionTime: Long = maxIdleStateRetentionTime - - override def setIdleStateRetentionTime(minTime: Time, maxTime: Time): Unit = { - super.setIdleStateRetention(Duration.ofMillis(minTime.toMilliseconds)) - minIdleStateRetentionTime = minTime.toMilliseconds - maxIdleStateRetentionTime = maxTime.toMilliseconds - } - } } object HarnessTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/OverAggregateHarnessTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/OverAggregateHarnessTest.scala index 7c1f566f02a9f..e2d280434bc1e 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/OverAggregateHarnessTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/OverAggregateHarnessTest.scala @@ -38,6 +38,7 @@ import org.junit.runners.Parameterized import org.junit.{Before, Test} import java.lang.{Long => JLong} +import java.time.Duration import java.util.concurrent.ConcurrentLinkedQueue import scala.collection.mutable @@ -49,8 +50,7 @@ class OverAggregateHarnessTest(mode: StateBackendMode) extends HarnessTestBase(m override def before(): Unit = { super.before() val setting = EnvironmentSettings.newInstance().inStreamingMode().build() - val config = new TestTableConfig - this.tEnv = StreamTableEnvironmentImpl.create(env, setting, config) + this.tEnv = StreamTableEnvironmentImpl.create(env, setting) } @Test @@ -127,9 +127,9 @@ class OverAggregateHarnessTest(mode: StateBackendMode) extends HarnessTestBase(m expectedOutput.add(new StreamRecord( row(2L: JLong, "aaa", 8L: JLong, null, 7L: JLong, 8L: JLong))) expectedOutput.add(new StreamRecord( - row(2L: JLong, "aaa", 9L: JLong, null, 8L: JLong, 9L: JLong))) + row(2L: JLong, "aaa", 10L: JLong, null, 10L: JLong, 10L: JLong))) expectedOutput.add(new StreamRecord( - row(2L: JLong, "aaa", 10L: JLong, null, 9L: JLong, 10L: JLong))) + row(2L: JLong, "aaa", 9L: JLong, null, 8L: JLong, 9L: JLong))) expectedOutput.add(new StreamRecord( row(2L: JLong, "bbb", 40L: JLong, null, 40L: JLong, 40L: JLong))) @@ -155,7 +155,7 @@ class OverAggregateHarnessTest(mode: StateBackendMode) extends HarnessTestBase(m """.stripMargin val t1 = tEnv.sqlQuery(sql) - tEnv.getConfig.setIdleStateRetentionTime(Time.seconds(2), Time.seconds(4)) + tEnv.getConfig.setIdleStateRetention(Duration.ofSeconds(2)) val testHarness = createHarnessTester(t1.toAppendStream[Row], "OverAggregate") val outputType = Array( DataTypes.BIGINT().getLogicalType, @@ -306,7 +306,7 @@ class OverAggregateHarnessTest(mode: StateBackendMode) extends HarnessTestBase(m """.stripMargin val t1 = tEnv.sqlQuery(sql) - tEnv.getConfig.setIdleStateRetentionTime(Time.seconds(2), Time.seconds(4)) + tEnv.getConfig.setIdleStateRetention(Duration.ofSeconds(2)) val testHarness = createHarnessTester(t1.toAppendStream[Row], "OverAggregate") val assertor = new RowDataHarnessAssertor( Array( @@ -533,7 +533,7 @@ class OverAggregateHarnessTest(mode: StateBackendMode) extends HarnessTestBase(m """.stripMargin val t1 = tEnv.sqlQuery(sql) - tEnv.getConfig.setIdleStateRetentionTime(Time.seconds(1), Time.seconds(2)) + tEnv.getConfig.setIdleStateRetention(Duration.ofSeconds(1)) val testHarness = createHarnessTester(t1.toAppendStream[Row], "OverAggregate") val assertor = new RowDataHarnessAssertor( Array( @@ -685,7 +685,7 @@ class OverAggregateHarnessTest(mode: StateBackendMode) extends HarnessTestBase(m """.stripMargin val t1 = tEnv.sqlQuery(sql) - tEnv.getConfig.setIdleStateRetentionTime(Time.seconds(1), Time.seconds(2)) + tEnv.getConfig.setIdleStateRetention(Duration.ofSeconds(1)) val testHarness = createHarnessTester(t1.toAppendStream[Row], "OverAggregate") val assertor = new RowDataHarnessAssertor( Array( @@ -827,7 +827,7 @@ class OverAggregateHarnessTest(mode: StateBackendMode) extends HarnessTestBase(m """.stripMargin val t1 = tEnv.sqlQuery(sql) - tEnv.getConfig.setIdleStateRetentionTime(Time.seconds(1), Time.seconds(2)) + tEnv.getConfig.setIdleStateRetention(Duration.ofSeconds(1)) val testHarness = createHarnessTester(t1.toAppendStream[Row], "OverAggregate") val assertor = new RowDataHarnessAssertor( Array( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/RankHarnessTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/RankHarnessTest.scala index 3c97370339d4c..14724a6dc7243 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/RankHarnessTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/RankHarnessTest.scala @@ -49,8 +49,7 @@ class RankHarnessTest(mode: StateBackendMode) extends HarnessTestBase(mode) { override def before(): Unit = { super.before() val setting = EnvironmentSettings.newInstance().inStreamingMode().build() - val config = new TestTableConfig - this.tEnv = StreamTableEnvironmentImpl.create(env, setting, config) + this.tEnv = StreamTableEnvironmentImpl.create(env, setting) } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/TableAggregateHarnessTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/TableAggregateHarnessTest.scala index d9dcf7fbc5745..8a4378d66ed8f 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/TableAggregateHarnessTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/TableAggregateHarnessTest.scala @@ -49,8 +49,7 @@ class TableAggregateHarnessTest(mode: StateBackendMode) extends HarnessTestBase( override def before(): Unit = { super.before() val setting = EnvironmentSettings.newInstance().inStreamingMode().build() - val config = new TestTableConfig - this.tEnv = StreamTableEnvironmentImpl.create(env, setting, config) + this.tEnv = StreamTableEnvironmentImpl.create(env, setting) } val data = new mutable.MutableList[(Int, Int)] diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/GroupWindowITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/GroupWindowITCase.scala index 2df538e26836c..05b8a5a72388b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/GroupWindowITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/GroupWindowITCase.scala @@ -583,9 +583,8 @@ class GroupWindowITCase(mode: StateBackendMode, useTimestampLtz: Boolean) private def withLateFireDelay(tableConfig: TableConfig, interval: Time): Unit = { val intervalInMillis = interval.toMilliseconds - val lateFireDelay: Duration = tableConfig.getConfiguration - .getOptional(TABLE_EXEC_EMIT_LATE_FIRE_DELAY) - .orElse(null) + val lateFireDelay: Duration = + tableConfig.getOptional(TABLE_EXEC_EMIT_LATE_FIRE_DELAY).orElse(null) if (lateFireDelay != null && (lateFireDelay.toMillis != intervalInMillis)) { // lateFireInterval of the two query config is not equal and not the default throw new RuntimeException( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/StreamTableEnvironmentITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/StreamTableEnvironmentITCase.scala index 1282aba6cdb13..939d2c357e5c0 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/StreamTableEnvironmentITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/StreamTableEnvironmentITCase.scala @@ -19,11 +19,20 @@ package org.apache.flink.table.planner.runtime.stream.sql import org.apache.flink.api.scala._ +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.datastream.DataStreamSource +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import org.apache.flink.streaming.api.scala.DataStream import org.apache.flink.table.api._ +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment +import org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl +import org.apache.flink.table.api.bridge.scala import org.apache.flink.table.api.bridge.scala._ +import org.apache.flink.table.api.config.TableConfigOptions import org.apache.flink.table.planner.runtime.utils.JavaPojos.{Device, Order, Person, ProductItem} import org.apache.flink.table.planner.runtime.utils.{StreamingTestBase, StringSink} +import org.assertj.core.api.Assertions.assertThat + import org.junit.Assert.assertEquals import org.junit.Test @@ -137,4 +146,20 @@ class StreamTableEnvironmentITCase extends StreamingTestBase { "(true,Order{user=1, product='Product{name='beer', id=10}', amount=3})") assertEquals(expected.sorted, sink.getResults.sorted) } + + @Test + def testTableConfigInheritsEnvironmentSettings(): Unit = { + val config = new Configuration + config.setString(TableConfigOptions.TABLE_CATALOG_NAME, "myCatalog") + val env = StreamExecutionEnvironment.getExecutionEnvironment(config) + val tEnv = StreamTableEnvironment.create(env) + assertThat(tEnv.getConfig.get(TableConfigOptions.TABLE_CATALOG_NAME)).isEqualTo("myCatalog") + + val scalaEnv = org.apache.flink.streaming.api.scala.StreamExecutionEnvironment + .getExecutionEnvironment + val scalaTEnv = scala.StreamTableEnvironment.create( + scalaEnv, EnvironmentSettings.newInstance.withConfiguration(config).build) + assertThat(scalaTEnv.getConfig.get(TableConfigOptions.TABLE_CATALOG_NAME)) + .isEqualTo("myCatalog") + } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSinkITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSinkITCase.scala index 20b61a3630ac8..a05687312cec8 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSinkITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSinkITCase.scala @@ -154,4 +154,40 @@ class TableSinkITCase(mode: StateBackendMode) extends StreamingWithStateTestBase val expected = List("+I[jason, 4]") assertEquals(expected.sorted, result.sorted) } + + @Test + def testInsertPartColumn(): Unit = { + tEnv.executeSql( + """ + |CREATE TABLE zm_test ( + | `person` String, + | `votes` BIGINT, + | `m1` MAP, + | `m2` MAP, + | `m3` MAP, + | `m4` MAP + |) WITH ( + | 'connector' = 'values', + | 'sink-insert-only' = 'true' + |) + |""".stripMargin) + + tEnv.executeSql( + """ + |insert into zm_test(`person`, `votes`) + | select + | `person`, + | `votes` + | from + | src + |""".stripMargin).await() + + val result = TestValuesTableFactory.getResults("zm_test") + val expected = List( + "+I[jason, 1, null, null, null, null]", + "+I[jason, 1, null, null, null, null]", + "+I[jason, 1, null, null, null, null]", + "+I[jason, 1, null, null, null, null]") + assertEquals(expected.sorted, result.sorted) + } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSourceITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSourceITCase.scala index 7ff10b2ccf81c..465db9cbe83c7 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSourceITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSourceITCase.scala @@ -76,6 +76,7 @@ class TableSourceITCase extends StreamingTestBase { |CREATE TABLE MetadataTable ( | `a` INT, | `other_metadata` INT METADATA FROM 'metadata_3', + | `other_metadata2` AS CAST(`other_metadata` AS BIGINT), | `b` BIGINT, | `metadata_1` INT METADATA, | `computed` AS `metadata_1` * 2, @@ -313,6 +314,22 @@ class TableSourceITCase extends StreamingTestBase { assertEquals(expected.sorted, sink.getAppendResults.sorted) } + @Test + def testDuplicateMetadataFromSameKey(): Unit = { + val result = tEnv.sqlQuery( + "SELECT other_metadata, other_metadata2, metadata_2 FROM MetadataTable") + .toAppendStream[Row] + val sink = new TestingAppendSink + result.addSink(sink) + env.execute() + + val expected = Seq( + "1,1,Hallo", + "1,1,Hallo Welt wie", + "2,2,Hallo Welt") + assertEquals(expected.sorted, sink.getAppendResults.sorted) + } + @Test def testNestedProjectionWithMetadataAccess(): Unit = { val query = diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingTestBase.scala index 9f106c4f25634..683543b09b2be 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingTestBase.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.planner.factories.TestValuesTableFactory import org.apache.flink.test.util.AbstractTestBase import org.apache.flink.types.Row +import org.junit.jupiter.api.{AfterEach, BeforeEach} import org.junit.rules.{ExpectedException, TemporaryFolder} import org.junit.{After, Before, Rule} @@ -44,6 +45,7 @@ class StreamingTestBase extends AbstractTestBase { def tempFolder: TemporaryFolder = _tempFolder @Before + @BeforeEach def before(): Unit = { this.env = StreamExecutionEnvironment.getExecutionEnvironment env.setParallelism(4) @@ -55,6 +57,7 @@ class StreamingTestBase extends AbstractTestBase { } @After + @AfterEach def after(): Unit = { StreamTestSink.clear() TestValuesTableFactory.clearAllData() diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingWithMiniBatchTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingWithMiniBatchTestBase.scala index 9324b9226a4cb..583c4cba549a6 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingWithMiniBatchTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingWithMiniBatchTestBase.scala @@ -39,9 +39,9 @@ abstract class StreamingWithMiniBatchTestBase( val tableConfig = tEnv.getConfig miniBatch match { case MiniBatchOn => - tableConfig.getConfiguration.setBoolean(TABLE_EXEC_MINIBATCH_ENABLED, true) - tableConfig.getConfiguration.set(TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)) - tableConfig.getConfiguration.setLong(TABLE_EXEC_MINIBATCH_SIZE, 3L) + tableConfig.set(TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(true)) + tableConfig.set(TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)) + tableConfig.set(TABLE_EXEC_MINIBATCH_SIZE, Long.box(3)) case MiniBatchOff => tableConfig.getConfiguration.removeConfig(TABLE_EXEC_MINIBATCH_ALLOW_LATENCY) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala index 6cb12cce545f8..83818a59819f2 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala @@ -26,7 +26,6 @@ import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation} import org.apache.flink.api.java.typeutils.{PojoTypeInfo, RowTypeInfo, TupleTypeInfo} import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo import org.apache.flink.configuration.ExecutionOptions - import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.{JsonNode, ObjectMapper} import org.apache.flink.streaming.api.datastream.DataStream import org.apache.flink.streaming.api.environment.{LocalStreamEnvironment, StreamExecutionEnvironment} @@ -1261,7 +1260,7 @@ case class StreamTableTestUtil( } def buildStreamProgram(firstProgramNameToRemove: String): Unit = { - val program = FlinkStreamProgram.buildProgram(tableEnv.getConfig.getConfiguration) + val program = FlinkStreamProgram.buildProgram(tableEnv.getConfig) var startRemove = false program.getProgramNames.foreach { name => @@ -1285,8 +1284,7 @@ case class StreamTableTestUtil( def getStreamProgram(): FlinkChainedProgram[StreamOptimizeContext] = { val tableConfig = tableEnv.getConfig val calciteConfig = TableConfigUtils.getCalciteConfig(tableConfig) - calciteConfig.getStreamProgram.getOrElse(FlinkStreamProgram.buildProgram( - tableConfig.getConfiguration)) + calciteConfig.getStreamProgram.getOrElse(FlinkStreamProgram.buildProgram(tableConfig)) } def enableMiniBatch(): Unit = { @@ -1346,7 +1344,7 @@ case class BatchTableTestUtil( extends TableTestUtil(test, isStreamingMode = false, catalogManager, conf) { def buildBatchProgram(firstProgramNameToRemove: String): Unit = { - val program = FlinkBatchProgram.buildProgram(tableEnv.getConfig.getConfiguration) + val program = FlinkBatchProgram.buildProgram(tableEnv.getConfig) var startRemove = false program.getProgramNames.foreach { name => @@ -1370,8 +1368,7 @@ case class BatchTableTestUtil( def getBatchProgram(): FlinkChainedProgram[BatchOptimizeContext] = { val tableConfig = tableEnv.getConfig val calciteConfig = TableConfigUtils.getCalciteConfig(tableConfig) - calciteConfig.getBatchProgram.getOrElse(FlinkBatchProgram.buildProgram( - tableConfig.getConfiguration)) + calciteConfig.getBatchProgram.getOrElse(FlinkBatchProgram.buildProgram(tableConfig)) } def createCollectTableSink( @@ -1526,11 +1523,18 @@ object TestingTableEnvironment { catalogManager: Option[CatalogManager] = None, tableConfig: TableConfig): TestingTableEnvironment = { - tableConfig.addConfiguration(settings.toConfiguration) - // temporary solution until FLINK-15635 is fixed val classLoader = Thread.currentThread.getContextClassLoader + val executorFactory = FactoryUtil.discoverFactory( + classLoader, classOf[ExecutorFactory], ExecutorFactory.DEFAULT_IDENTIFIER) + + val executor = executorFactory.create(settings.getConfiguration) + + tableConfig.setRootConfiguration(executor.getConfiguration) + tableConfig.addConfiguration(settings.getConfiguration) + + val moduleManager = new ModuleManager val catalogMgr = catalogManager match { @@ -1538,7 +1542,7 @@ object TestingTableEnvironment { case _ => CatalogManager.newBuilder .classLoader(classLoader) - .config(tableConfig.getConfiguration) + .config(tableConfig) .defaultCatalog( settings.getBuiltInCatalogName, new GenericInMemoryCatalog( @@ -1547,14 +1551,10 @@ object TestingTableEnvironment { .build } - val functionCatalog = new FunctionCatalog(tableConfig, catalogMgr, moduleManager) - - val executorFactory = - FactoryUtil.discoverFactory(classLoader, classOf[ExecutorFactory], settings.getExecutor) - val executor = executorFactory.create(tableConfig.getConfiguration) + val functionCatalog = new FunctionCatalog(settings.getConfiguration, catalogMgr, moduleManager) - val planner = PlannerFactoryUtil.createPlanner(settings.getPlanner, executor, tableConfig, - moduleManager, catalogMgr, functionCatalog).asInstanceOf[PlannerBase] + val planner = PlannerFactoryUtil.createPlanner( + executor, tableConfig, moduleManager, catalogMgr, functionCatalog).asInstanceOf[PlannerBase] new TestingTableEnvironment( catalogMgr, diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/UserDefinedTableFunctions.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/UserDefinedTableFunctions.scala index f087d5a91dd7b..6fcad3d50c817 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/UserDefinedTableFunctions.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/UserDefinedTableFunctions.scala @@ -29,8 +29,6 @@ import org.apache.flink.types.Row import org.junit.Assert -import java.lang.Boolean - import scala.annotation.varargs @@ -117,9 +115,10 @@ class TableFunc3(data: String, conf: Map[String, String]) extends TableFunction[ } @SerialVersionUID(1L) +@DataTypeHint("ROW") class MockPythonTableFunction extends TableFunction[Row] with PythonFunction { - def eval(x: Int, y: Int) = ??? + def eval(x: java.lang.Integer, y: java.lang.Integer) = ??? override def getResultType: TypeInformation[Row] = new RowTypeInfo(Types.INT, Types.INT) @@ -368,28 +367,19 @@ class MockPythonTableFunction extends TableFunction[Row] with PythonFunction { //} @SerialVersionUID(1L) +@DataTypeHint("ROW") class TableFunc4 extends TableFunction[Row] { def eval(b: Byte, s: Short, f: Float): Unit = { collect(Row.of("Byte=" + b, "Short=" + s, "Float=" + f)) } - - override def getResultType: TypeInformation[Row] = { - new RowTypeInfo(Types.STRING, Types.STRING, Types.STRING) - } } @SerialVersionUID(1L) +@DataTypeHint("ROW") class TableFunc6 extends TableFunction[Row] { - def eval(row: Row): Unit = { + def eval(@DataTypeHint("ROW") row: Row): Unit = { collect(row) } - - override def getParameterTypes(signature: Array[Class[_]]): Array[TypeInformation[_]] = - Array(new RowTypeInfo(Types.INT, Types.INT, Types.INT)) - - override def getResultType: TypeInformation[Row] = { - new RowTypeInfo(Types.INT, Types.INT, Types.INT) - } } @SerialVersionUID(1L) @@ -421,12 +411,12 @@ class VarArgsFunc0 extends TableFunction[String] { } @SerialVersionUID(1L) -class HierarchyTableFunction extends SplittableTableFunction[Boolean, Integer] { +class HierarchyTableFunction extends SplittableTableFunction[java.lang.Boolean, Integer] { def eval(user: String) { if (user.contains("#")) { val splits = user.split("#") val age = splits(1).toInt - collect(new Tuple3[String, Boolean, Integer](splits(0), age >= 20, age)) + collect(new Tuple3[String, java.lang.Boolean, Integer](splits(0), age >= 20, age)) } } } diff --git a/flink-table/flink-table-runtime/pom.xml b/flink-table/flink-table-runtime/pom.xml index a846bf9857dca..9681d923bb2d8 100644 --- a/flink-table/flink-table-runtime/pom.xml +++ b/flink-table/flink-table-runtime/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/aggregate/JsonObjectAggFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/aggregate/JsonObjectAggFunction.java index 8152f988fe32a..4be69c9259266 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/aggregate/JsonObjectAggFunction.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/aggregate/JsonObjectAggFunction.java @@ -57,6 +57,7 @@ public class JsonObjectAggFunction extends BuiltInAggregateFunction { private static final long serialVersionUID = 1L; + private static final StringData NULL_STRING_DATA = StringData.fromBytes(new byte[] {}); private static final NullNode NULL_NODE = getNodeFactory().nullNode(); private final transient List argumentTypes; @@ -107,7 +108,9 @@ public void accumulate(Accumulator acc, StringData keyData, @Nullable StringData if (valueData == null) { if (!skipNulls) { - acc.map.put(keyData, null); + // We cannot use null for StringData here, since it's not supported by the + // StringDataSerializer, instead use a StringData with an empty byte[] + acc.map.put(keyData, NULL_STRING_DATA); } } else { acc.map.put(keyData, valueData); @@ -135,7 +138,7 @@ public String getValue(Accumulator acc) { for (final StringData key : acc.map.keys()) { final StringData value = acc.map.get(key); final JsonNode valueNode = - value == null + value.toBytes().length == 0 ? NULL_NODE : getNodeFactory().rawValueNode(new RawValue(value.toString())); diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/generated/GeneratedWatermarkGeneratorSupplier.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/generated/GeneratedWatermarkGeneratorSupplier.java index 583b3c2680c1a..11fdc4a10af83 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/generated/GeneratedWatermarkGeneratorSupplier.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/generated/GeneratedWatermarkGeneratorSupplier.java @@ -39,12 +39,10 @@ public class GeneratedWatermarkGeneratorSupplier implements WatermarkGeneratorSupplier { private static final long serialVersionUID = 1L; - private final Configuration configuration; private final GeneratedWatermarkGenerator generatedWatermarkGenerator; public GeneratedWatermarkGeneratorSupplier( - Configuration configuration, GeneratedWatermarkGenerator generatedWatermarkGenerator) { - this.configuration = configuration; + GeneratedWatermarkGenerator generatedWatermarkGenerator) { this.generatedWatermarkGenerator = generatedWatermarkGenerator; } @@ -60,12 +58,11 @@ public GeneratedWatermarkGeneratorSupplier( new GeneratedWatermarkGenerator( generatedWatermarkGenerator.getClassName(), generatedWatermarkGenerator.getCode(), - references.toArray(), - configuration) + references.toArray()) .newInstance(Thread.currentThread().getContextClassLoader()); try { - innerWatermarkGenerator.open(configuration); + innerWatermarkGenerator.open(new Configuration()); } catch (Exception e) { throw new RuntimeException("Fail to instantiate generated watermark generator.", e); } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/TimeWindow.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/TimeWindow.java index e358c994d8ea4..11e2ed3fd1887 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/TimeWindow.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/TimeWindow.java @@ -220,7 +220,13 @@ public TimeWindowSerializerSnapshot() { * @return window start */ public static long getWindowStartWithOffset(long timestamp, long offset, long windowSize) { - return timestamp - (timestamp - offset + windowSize) % windowSize; + final long remainder = (timestamp - offset) % windowSize; + // handle both positive and negative cases + if (remainder < 0) { + return timestamp - (remainder + windowSize); + } else { + return timestamp - remainder; + } } public static TimeWindow of(long start, long end) { diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/typeutils/ArrayDataSerializer.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/typeutils/ArrayDataSerializer.java index 078f01a46d3d3..26a6b0bdea1c1 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/typeutils/ArrayDataSerializer.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/typeutils/ArrayDataSerializer.java @@ -84,10 +84,12 @@ public ArrayData createInstance() { public ArrayData copy(ArrayData from) { if (from instanceof GenericArrayData) { return copyGenericArray((GenericArrayData) from); + } else if (from instanceof ColumnarArrayData) { + return copyColumnarArray((ColumnarArrayData) from); } else if (from instanceof BinaryArrayData) { return ((BinaryArrayData) from).copy(); } else { - return copyColumnarArray((ColumnarArrayData) from); + return toBinaryArray(from); } } diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/typeutils/ArrayDataSerializerTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/typeutils/ArrayDataSerializerTest.java index 558bbbb0ca281..29fcbda37f29f 100644 --- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/typeutils/ArrayDataSerializerTest.java +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/typeutils/ArrayDataSerializerTest.java @@ -29,6 +29,7 @@ import org.apache.flink.table.data.writer.BinaryArrayWriter; import org.apache.flink.testutils.DeeplyEqualsChecker; +import java.lang.reflect.Proxy; import java.nio.charset.StandardCharsets; /** A test for the {@link ArrayDataSerializer}. */ @@ -88,6 +89,7 @@ protected ArrayData[] getTestData() { createArray("11", "haa", "ke"), createArray("11", "lele", "haa", "ke"), createColumnarArray("11", "lele", "haa", "ke"), + createCustomTypeArray("11", "lele", "haa", "ke"), }; } @@ -108,4 +110,14 @@ private static ColumnarArrayData createColumnarArray(String... vs) { } return new ColumnarArrayData(vector, 0, vs.length); } + + static ArrayData createCustomTypeArray(String... vs) { + BinaryArrayData binaryArrayData = createArray(vs); + Object customArrayData = + Proxy.newProxyInstance( + ArrayDataSerializerTest.class.getClassLoader(), + new Class[] {ArrayData.class}, + (proxy, method, args) -> method.invoke(binaryArrayData, args)); + return (ArrayData) customArrayData; + } } diff --git a/flink-table/flink-table-test-utils/pom.xml b/flink-table/flink-table-test-utils/pom.xml index 28f14ad4a68fe..537bd696d0b3b 100644 --- a/flink-table/flink-table-test-utils/pom.xml +++ b/flink-table/flink-table-test-utils/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/pom.xml b/flink-table/pom.xml index b3d66fd5e422e..9c66bdb5c5f58 100644 --- a/flink-table/pom.xml +++ b/flink-table/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-test-utils-parent/flink-connector-test-utils/pom.xml b/flink-test-utils-parent/flink-connector-test-utils/pom.xml index 13aeea784b55b..eb989a86311db 100644 --- a/flink-test-utils-parent/flink-connector-test-utils/pom.xml +++ b/flink-test-utils-parent/flink-connector-test-utils/pom.xml @@ -25,7 +25,7 @@ org.apache.flink flink-test-utils-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/environment/MiniClusterTestEnvironment.java b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/environment/MiniClusterTestEnvironment.java index 0ecbe7f085c6c..e68560ada54b0 100644 --- a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/environment/MiniClusterTestEnvironment.java +++ b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/environment/MiniClusterTestEnvironment.java @@ -19,7 +19,6 @@ package org.apache.flink.connector.testframe.environment; import org.apache.flink.annotation.Experimental; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.JobClient; import org.apache.flink.runtime.highavailability.nonha.embedded.HaLeadershipControl; @@ -38,7 +37,6 @@ import java.net.URI; import java.nio.file.Files; import java.nio.file.Path; -import java.time.Duration; import java.util.Collections; import java.util.Optional; import java.util.concurrent.ExecutionException; @@ -132,8 +130,7 @@ public void triggerTaskManagerFailover(JobClient jobClient, Runnable afterFailAc throws Exception { terminateTaskManager(); CommonTestUtils.waitForNoTaskRunning( - () -> miniCluster.getRestClusterClient().getJobDetails(jobClient.getJobID()).get(), - Deadline.fromNow(Duration.ofMinutes(5))); + () -> miniCluster.getRestClusterClient().getJobDetails(jobClient.getJobID()).get()); afterFailAction.run(); startTaskManager(); } diff --git a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/source/enumerator/NoOpEnumStateSerializer.java b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/source/enumerator/NoOpEnumStateSerializer.java index 7be0e8d2d88d9..0628e632f19b3 100644 --- a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/source/enumerator/NoOpEnumStateSerializer.java +++ b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/source/enumerator/NoOpEnumStateSerializer.java @@ -22,7 +22,7 @@ import java.io.IOException; -/** Mock enumerator state seializer. */ +/** Mock enumerator state serializer. */ public class NoOpEnumStateSerializer implements SimpleVersionedSerializer { @Override public int getVersion() { diff --git a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/testsuites/SinkTestSuiteBase.java b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/testsuites/SinkTestSuiteBase.java index e958e70f71f23..df30164097fae 100644 --- a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/testsuites/SinkTestSuiteBase.java +++ b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/testsuites/SinkTestSuiteBase.java @@ -23,7 +23,6 @@ import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.api.connector.source.Boundedness; @@ -80,7 +79,6 @@ import java.util.stream.Collectors; import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT; -import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_JOB_STATUS_CHANGE_TIMEOUT; import static org.apache.flink.connector.testframe.utils.MetricQuerier.getJobDetails; import static org.apache.flink.runtime.testutils.CommonTestUtils.terminateJob; import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForAllTaskRunning; @@ -152,10 +150,7 @@ public void testBasicSink( .name("sinkInSinkTest"); final JobClient jobClient = execEnv.executeAsync("DataStream Sink Test"); - waitForJobStatus( - jobClient, - Collections.singletonList(JobStatus.FINISHED), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + waitForJobStatus(jobClient, Collections.singletonList(JobStatus.FINISHED)); // Check test result checkResultWithSemantic( @@ -281,8 +276,7 @@ private void restartFromSavepoint( getJobDetails( new RestClient(new Configuration(), executorService), testEnv.getRestEndpoint(), - jobClient.getJobID()), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + jobClient.getJobID())); waitExpectedSizeData(iterator, numBeforeSuccess); @@ -291,10 +285,7 @@ private void restartFromSavepoint( .stopWithSavepoint( true, testEnv.getCheckpointUri(), SavepointFormatType.CANONICAL) .get(30, TimeUnit.SECONDS); - waitForJobStatus( - jobClient, - Collections.singletonList(JobStatus.FINISHED), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + waitForJobStatus(jobClient, Collections.singletonList(JobStatus.FINISHED)); } catch (Exception e) { executorService.shutdown(); killJob(jobClient); @@ -392,8 +383,7 @@ public void testMetrics( getJobDetails( new RestClient(new Configuration(), executorService), testEnv.getRestEndpoint(), - jobClient.getJobID()), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + jobClient.getJobID())); waitUntilCondition( () -> { @@ -411,8 +401,7 @@ public void testMetrics( // skip failed assert try return false; } - }, - Deadline.fromNow(DEFAULT_COLLECT_DATA_TIMEOUT)); + }); } finally { // Clean up executorService.shutdown(); @@ -521,8 +510,7 @@ private void checkResultWithSemantic( } catch (Throwable t) { return false; } - }, - Deadline.fromNow(DEFAULT_COLLECT_DATA_TIMEOUT)); + }); } /** Compare the metrics. */ @@ -562,10 +550,7 @@ private TestingSinkSettings getTestingSinkSettings(CheckpointingMode checkpointi private void killJob(JobClient jobClient) throws Exception { terminateJob(jobClient); - waitForJobStatus( - jobClient, - Collections.singletonList(JobStatus.CANCELED), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + waitForJobStatus(jobClient, Collections.singletonList(JobStatus.CANCELED)); } private DataStreamSink tryCreateSink( diff --git a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/testsuites/SourceTestSuiteBase.java b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/testsuites/SourceTestSuiteBase.java index a90d05bc135a0..b345b4114e53e 100644 --- a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/testsuites/SourceTestSuiteBase.java +++ b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/testsuites/SourceTestSuiteBase.java @@ -23,7 +23,6 @@ import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.Source; @@ -77,7 +76,6 @@ import java.util.concurrent.TimeUnit; import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT; -import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_JOB_STATUS_CHANGE_TIMEOUT; import static org.apache.flink.connector.testframe.utils.MetricQuerier.getJobDetails; import static org.apache.flink.runtime.testutils.CommonTestUtils.terminateJob; import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForAllTaskRunning; @@ -159,10 +157,7 @@ public void testSourceSingleSplit( } // Step 5: Clean up - waitForJobStatus( - jobClient, - Collections.singletonList(JobStatus.FINISHED), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + waitForJobStatus(jobClient, Collections.singletonList(JobStatus.FINISHED)); } /** @@ -346,10 +341,7 @@ private void restartFromSavepoint( .stopWithSavepoint( true, testEnv.getCheckpointUri(), SavepointFormatType.CANONICAL) .get(30, TimeUnit.SECONDS); - waitForJobStatus( - jobClient, - Collections.singletonList(JobStatus.FINISHED), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + waitForJobStatus(jobClient, Collections.singletonList(JobStatus.FINISHED)); // Step 5: Generate new test data final List> newTestRecordCollections = new ArrayList<>(); @@ -380,10 +372,7 @@ private void restartFromSavepoint( final JobClient restartJobClient = restartEnv.executeAsync("Restart Test"); - waitForJobStatus( - restartJobClient, - Collections.singletonList(JobStatus.RUNNING), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + waitForJobStatus(restartJobClient, Collections.singletonList(JobStatus.RUNNING)); try { iterator.setJobClient(restartJobClient); @@ -458,8 +447,7 @@ public void testSourceMetrics( getJobDetails( new RestClient(new Configuration(), executorService), testEnv.getRestEndpoint(), - jobClient.getJobID()), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + jobClient.getJobID())); waitUntilCondition( () -> { @@ -475,8 +463,7 @@ public void testSourceMetrics( // skip failed assert try return false; } - }, - Deadline.fromNow(DEFAULT_COLLECT_DATA_TIMEOUT)); + }); } finally { // Clean up executorService.shutdown(); @@ -539,10 +526,7 @@ public void testIdleReader( } // Step 5: Clean up - waitForJobStatus( - jobClient, - Collections.singletonList(JobStatus.FINISHED), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + waitForJobStatus(jobClient, Collections.singletonList(JobStatus.FINISHED)); } /** @@ -614,10 +598,7 @@ public void testTaskManagerFailure( controller.triggerTaskManagerFailover(jobClient, () -> {}); LOG.info("Waiting for job recovering from failure"); - waitForJobStatus( - jobClient, - Collections.singletonList(JobStatus.RUNNING), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + waitForJobStatus(jobClient, Collections.singletonList(JobStatus.RUNNING)); // Step 6: Write test data again to external system List testRecordsAfterFailure = @@ -639,10 +620,7 @@ public void testTaskManagerFailure( // Step 8: Clean up terminateJob(jobClient); - waitForJobStatus( - jobClient, - Collections.singletonList(JobStatus.CANCELED), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + waitForJobStatus(jobClient, Collections.singletonList(JobStatus.CANCELED)); iterator.close(); } @@ -790,10 +768,7 @@ private boolean checkSourceMetrics( private void killJob(JobClient jobClient) throws Exception { terminateJob(jobClient); - waitForJobStatus( - jobClient, - Collections.singletonList(JobStatus.CANCELED), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + waitForJobStatus(jobClient, Collections.singletonList(JobStatus.CANCELED)); } /** Builder class for constructing {@link CollectResultIterator} of collect sink. */ diff --git a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/utils/ConnectorTestConstants.java b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/utils/ConnectorTestConstants.java index f9a83195c4166..88c46c2851796 100644 --- a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/utils/ConnectorTestConstants.java +++ b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/utils/ConnectorTestConstants.java @@ -26,6 +26,5 @@ public class ConnectorTestConstants { public static final long SLOT_REQUEST_TIMEOUT_MS = 10_000L; public static final long HEARTBEAT_TIMEOUT_MS = 5_000L; public static final long HEARTBEAT_INTERVAL_MS = 1000L; - public static final Duration DEFAULT_JOB_STATUS_CHANGE_TIMEOUT = Duration.ofSeconds(30L); public static final Duration DEFAULT_COLLECT_DATA_TIMEOUT = Duration.ofSeconds(120L); } diff --git a/flink-test-utils-parent/flink-test-utils-junit/pom.xml b/flink-test-utils-parent/flink-test-utils-junit/pom.xml index 85cab96d721e2..0a50f3fe20760 100644 --- a/flink-test-utils-parent/flink-test-utils-junit/pom.xml +++ b/flink-test-utils-parent/flink-test-utils-junit/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-test-utils-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/util/DockerImageVersions.java b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/util/DockerImageVersions.java index b8422f9d19efd..ec769037dcac5 100644 --- a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/util/DockerImageVersions.java +++ b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/util/DockerImageVersions.java @@ -42,7 +42,7 @@ public class DockerImageVersions { public static final String LOCALSTACK = "localstack/localstack:0.13.3"; - public static final String PULSAR = "apachepulsar/pulsar:2.9.1"; + public static final String PULSAR = "apachepulsar/pulsar:2.10.0"; public static final String CASSANDRA_3 = "cassandra:3.0"; diff --git a/flink-test-utils-parent/flink-test-utils/pom.xml b/flink-test-utils-parent/flink-test-utils/pom.xml index 06b1872f5e973..945ce88755f8c 100644 --- a/flink-test-utils-parent/flink-test-utils/pom.xml +++ b/flink-test-utils-parent/flink-test-utils/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-test-utils-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-test-utils-parent/pom.xml b/flink-test-utils-parent/pom.xml index 812dd6bcb8a45..ef5712dc5dbc0 100644 --- a/flink-test-utils-parent/pom.xml +++ b/flink-test-utils-parent/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml index 1acd328ea7e19..aa25f3edcb87f 100644 --- a/flink-tests/pom.xml +++ b/flink-tests/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointITCase.java deleted file mode 100644 index b763cad5295de..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointITCase.java +++ /dev/null @@ -1,410 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.jobmaster; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.JobStatus; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Deadline; -import org.apache.flink.api.common.time.Time; -import org.apache.flink.client.program.MiniClusterClient; -import org.apache.flink.core.execution.SavepointFormatType; -import org.apache.flink.core.testutils.OneShotLatch; -import org.apache.flink.runtime.checkpoint.CheckpointException; -import org.apache.flink.runtime.checkpoint.CheckpointFailureReason; -import org.apache.flink.runtime.checkpoint.CheckpointMetaData; -import org.apache.flink.runtime.checkpoint.CheckpointOptions; -import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy; -import org.apache.flink.runtime.checkpoint.SavepointType; -import org.apache.flink.runtime.checkpoint.SnapshotType; -import org.apache.flink.runtime.execution.Environment; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.JobGraphBuilder; -import org.apache.flink.runtime.jobgraph.JobVertex; -import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; -import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; -import org.apache.flink.runtime.jobgraph.tasks.TaskInvokable; -import org.apache.flink.runtime.testutils.CommonTestUtils; -import org.apache.flink.streaming.runtime.tasks.StreamTask; -import org.apache.flink.streaming.runtime.tasks.StreamTaskTest.NoOpStreamTask; -import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; -import org.apache.flink.test.util.AbstractTestBase; -import org.apache.flink.util.ExceptionUtils; - -import org.junit.Assume; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.time.Duration; -import java.util.Collections; -import java.util.Optional; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.either; -import static org.hamcrest.Matchers.equalTo; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -/** - * ITCases testing the stop with savepoint functionality. This includes checking both SUSPEND and - * TERMINATE. - */ -public class JobMasterStopWithSavepointITCase extends AbstractTestBase { - - @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - - private static final long CHECKPOINT_INTERVAL = 10; - private static final int PARALLELISM = 2; - - private static OneShotLatch finishingLatch; - - private static CountDownLatch invokeLatch; - - private static CountDownLatch numberOfRestarts; - private static final AtomicLong syncSavepointId = new AtomicLong(); - private static volatile CountDownLatch checkpointsToWaitFor; - - private Path savepointDirectory; - private MiniClusterClient clusterClient; - - private JobGraph jobGraph; - - @Test - public void throwingExceptionOnCallbackWithNoRestartsShouldFailTheSuspend() throws Exception { - throwingExceptionOnCallbackWithoutRestartsHelper(false); - } - - @Test - public void throwingExceptionOnCallbackWithNoRestartsShouldFailTheTerminate() throws Exception { - throwingExceptionOnCallbackWithoutRestartsHelper(true); - } - - private void throwingExceptionOnCallbackWithoutRestartsHelper(final boolean terminate) - throws Exception { - setUpJobGraph(ExceptionOnCallbackStreamTask.class, RestartStrategies.noRestart()); - - assertThat(getJobStatus(), equalTo(JobStatus.RUNNING)); - - try { - stopWithSavepoint(terminate).get(); - fail(); - } catch (Exception e) { - } - - // verifying that we actually received a synchronous checkpoint - assertTrue(syncSavepointId.get() > 0); - assertThat( - getJobStatus(), either(equalTo(JobStatus.FAILED)).or(equalTo(JobStatus.FAILING))); - } - - @Test - public void throwingExceptionOnCallbackWithRestartsShouldSimplyRestartInSuspend() - throws Exception { - throwingExceptionOnCallbackWithRestartsHelper(false); - } - - @Test - public void throwingExceptionOnCallbackWithRestartsShouldSimplyRestartInTerminate() - throws Exception { - throwingExceptionOnCallbackWithRestartsHelper(true); - } - - private void throwingExceptionOnCallbackWithRestartsHelper(final boolean terminate) - throws Exception { - final Deadline deadline = Deadline.fromNow(Duration.ofSeconds(15)); - final int numberOfCheckpointsToExpect = 10; - - numberOfRestarts = new CountDownLatch(2); - checkpointsToWaitFor = new CountDownLatch(numberOfCheckpointsToExpect); - - setUpJobGraph( - ExceptionOnCallbackStreamTask.class, - RestartStrategies.fixedDelayRestart(15, Time.milliseconds(10))); - assertThat(getJobStatus(), equalTo(JobStatus.RUNNING)); - try { - stopWithSavepoint(terminate).get(50, TimeUnit.MILLISECONDS); - fail(); - } catch (Exception e) { - // expected - } - - // wait until we restart at least 2 times and until we see at least 10 checkpoints. - assertTrue(numberOfRestarts.await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); - assertTrue( - checkpointsToWaitFor.await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); - - // verifying that we actually received a synchronous checkpoint - assertTrue(syncSavepointId.get() > 0); - - assertThat(getJobStatus(), equalTo(JobStatus.RUNNING)); - - // make sure that we saw the synchronous savepoint and - // that after that we saw more checkpoints due to restarts. - final long syncSavepoint = syncSavepointId.get(); - assertTrue(syncSavepoint > 0 && syncSavepoint < numberOfCheckpointsToExpect); - - clusterClient.cancel(jobGraph.getJobID()).get(); - assertThat( - getJobStatus(), - either(equalTo(JobStatus.CANCELLING)).or(equalTo(JobStatus.CANCELED))); - } - - @Test - public void testRestartCheckpointCoordinatorIfStopWithSavepointFails() throws Exception { - setUpJobGraph(CheckpointCountingTask.class, RestartStrategies.noRestart()); - - try { - Files.setPosixFilePermissions(savepointDirectory, Collections.emptySet()); - } catch (IOException e) { - Assume.assumeNoException(e); - } - - try { - stopWithSavepoint(true).get(); - fail(); - } catch (Exception e) { - Optional checkpointExceptionOptional = - ExceptionUtils.findThrowable(e, CheckpointException.class); - if (!checkpointExceptionOptional.isPresent()) { - throw e; - } - String exceptionMessage = checkpointExceptionOptional.get().getMessage(); - assertTrue( - "Stop with savepoint failed because of another cause " + exceptionMessage, - exceptionMessage.contains(CheckpointFailureReason.IO_EXCEPTION.message())); - } - - final JobStatus jobStatus = - clusterClient.getJobStatus(jobGraph.getJobID()).get(60, TimeUnit.SECONDS); - assertThat(jobStatus, equalTo(JobStatus.RUNNING)); - // assert that checkpoints are continued to be triggered - checkpointsToWaitFor = new CountDownLatch(1); - assertTrue(checkpointsToWaitFor.await(60L, TimeUnit.SECONDS)); - } - - private CompletableFuture stopWithSavepoint(boolean terminate) { - return MINI_CLUSTER_RESOURCE - .getMiniCluster() - .stopWithSavepoint( - jobGraph.getJobID(), - savepointDirectory.toAbsolutePath().toString(), - terminate, - SavepointFormatType.CANONICAL); - } - - private JobStatus getJobStatus() throws InterruptedException, ExecutionException { - return clusterClient.getJobStatus(jobGraph.getJobID()).get(); - } - - private void setUpJobGraph( - final Class invokable, - final RestartStrategies.RestartStrategyConfiguration restartStrategy) - throws Exception { - - finishingLatch = new OneShotLatch(); - - invokeLatch = new CountDownLatch(PARALLELISM); - - numberOfRestarts = new CountDownLatch(2); - checkpointsToWaitFor = new CountDownLatch(10); - - syncSavepointId.set(-1); - - savepointDirectory = temporaryFolder.newFolder().toPath(); - - Assume.assumeTrue( - "ClusterClient is not an instance of MiniClusterClient", - MINI_CLUSTER_RESOURCE.getClusterClient() instanceof MiniClusterClient); - - clusterClient = (MiniClusterClient) MINI_CLUSTER_RESOURCE.getClusterClient(); - - final ExecutionConfig config = new ExecutionConfig(); - config.setRestartStrategy(restartStrategy); - - final JobVertex vertex = new JobVertex("testVertex"); - vertex.setInvokableClass(invokable); - vertex.setParallelism(PARALLELISM); - - final JobCheckpointingSettings jobCheckpointingSettings = - new JobCheckpointingSettings( - new CheckpointCoordinatorConfiguration( - CHECKPOINT_INTERVAL, - 60_000, - 10, - 1, - CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION, - true, - false, - 0, - 0), - null); - - jobGraph = - JobGraphBuilder.newStreamingJobGraphBuilder() - .setExecutionConfig(config) - .addJobVertex(vertex) - .setJobCheckpointingSettings(jobCheckpointingSettings) - .build(); - - clusterClient.submitJob(jobGraph).get(); - assertTrue(invokeLatch.await(60, TimeUnit.SECONDS)); - waitForJob(); - } - - private void waitForJob() throws Exception { - Deadline deadline = Deadline.fromNow(Duration.ofMinutes(5)); - JobID jobID = jobGraph.getJobID(); - CommonTestUtils.waitForAllTaskRunning( - () -> - MINI_CLUSTER_RESOURCE - .getMiniCluster() - .getExecutionGraph(jobID) - .get(60, TimeUnit.SECONDS), - deadline, - false); - } - - /** - * A {@link StreamTask} which throws an exception in the {@code notifyCheckpointComplete()} for - * subtask 0. - */ - public static class ExceptionOnCallbackStreamTask extends CheckpointCountingTask { - - private long synchronousSavepointId = Long.MIN_VALUE; - - public ExceptionOnCallbackStreamTask(final Environment environment) throws Exception { - super(environment); - } - - @Override - protected void processInput(MailboxDefaultAction.Controller controller) throws Exception { - final long taskIndex = getEnvironment().getTaskInfo().getIndexOfThisSubtask(); - if (taskIndex == 0) { - numberOfRestarts.countDown(); - } - super.processInput(controller); - } - - @Override - public CompletableFuture triggerCheckpointAsync( - CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions) { - final long checkpointId = checkpointMetaData.getCheckpointId(); - final SnapshotType checkpointType = checkpointOptions.getCheckpointType(); - - if (checkpointType.isSavepoint() && ((SavepointType) checkpointType).isSynchronous()) { - synchronousSavepointId = checkpointId; - syncSavepointId.compareAndSet(-1, synchronousSavepointId); - } - - return super.triggerCheckpointAsync(checkpointMetaData, checkpointOptions); - } - - @Override - public Future notifyCheckpointCompleteAsync(long checkpointId) { - final long taskIndex = getEnvironment().getTaskInfo().getIndexOfThisSubtask(); - if (checkpointId == synchronousSavepointId && taskIndex == 0) { - throw new RuntimeException("Expected Exception"); - } - - return super.notifyCheckpointCompleteAsync(checkpointId); - } - - @Override - public Future notifyCheckpointAbortAsync( - long checkpointId, long latestCompletedCheckpointId) { - return CompletableFuture.completedFuture(null); - } - } - - /** A {@link StreamTask} that simply waits to be terminated normally. */ - public static class NoOpBlockingStreamTask extends NoOpStreamTask { - - private transient MailboxDefaultAction.Suspension suspension; - - public NoOpBlockingStreamTask(final Environment environment) throws Exception { - super(environment); - } - - @Override - protected void processInput(MailboxDefaultAction.Controller controller) throws Exception { - invokeLatch.countDown(); - if (suspension == null) { - suspension = controller.suspendDefaultAction(); - } else { - controller.suspendDefaultAction(); - mailboxProcessor.suspend(); - } - } - } - - /** - * A {@link StreamTask} that simply calls {@link CountDownLatch#countDown()} when invoking - * {@link #triggerCheckpointAsync}. - */ - public static class CheckpointCountingTask extends NoOpStreamTask { - - private transient MailboxDefaultAction.Suspension suspension; - - public CheckpointCountingTask(final Environment environment) throws Exception { - super(environment); - } - - @Override - protected void processInput(MailboxDefaultAction.Controller controller) throws Exception { - invokeLatch.countDown(); - if (suspension == null) { - suspension = controller.suspendDefaultAction(); - } else { - controller.suspendDefaultAction(); - mailboxProcessor.suspend(); - } - } - - @Override - protected void cancelTask() throws Exception { - super.cancelTask(); - if (suspension != null) { - suspension.resume(); - } - } - - @Override - public CompletableFuture triggerCheckpointAsync( - final CheckpointMetaData checkpointMetaData, - final CheckpointOptions checkpointOptions) { - final long taskIndex = getEnvironment().getTaskInfo().getIndexOfThisSubtask(); - if (taskIndex == 0) { - checkpointsToWaitFor.countDown(); - } - - return super.triggerCheckpointAsync(checkpointMetaData, checkpointOptions); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/runtime/operators/lifecycle/BoundedSourceITCase.java b/flink-tests/src/test/java/org/apache/flink/runtime/operators/lifecycle/BoundedSourceITCase.java index d03e9cbaa621e..6933fe63b9e47 100644 --- a/flink-tests/src/test/java/org/apache/flink/runtime/operators/lifecycle/BoundedSourceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/runtime/operators/lifecycle/BoundedSourceITCase.java @@ -17,19 +17,27 @@ package org.apache.flink.runtime.operators.lifecycle; +import org.apache.flink.changelog.fs.FsStateChangelogStorageFactory; +import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.operators.lifecycle.event.CheckpointCompletedEvent; import org.apache.flink.runtime.operators.lifecycle.graph.TestJobBuilders.TestingGraphBuilder; import org.apache.flink.runtime.operators.lifecycle.validation.DrainingValidator; import org.apache.flink.runtime.operators.lifecycle.validation.FinishingValidator; -import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.util.TestBaseUtils; import org.apache.flink.testutils.junit.SharedObjects; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameter; +import java.io.IOException; + import static org.apache.flink.runtime.operators.lifecycle.command.TestCommand.FINISH_SOURCES; import static org.apache.flink.runtime.operators.lifecycle.command.TestCommandDispatcher.TestCommandScope.ALL_SUBTASKS; import static org.apache.flink.runtime.operators.lifecycle.graph.TestJobBuilders.COMPLEX_GRAPH_BUILDER; @@ -46,10 +54,33 @@ * same. */ @RunWith(Parameterized.class) -public class BoundedSourceITCase extends AbstractTestBase { +public class BoundedSourceITCase extends TestBaseUtils { + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final MiniClusterWithClientResource miniClusterResource = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setConfiguration(configuration()) + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(4) + .build()); @Rule public final SharedObjects sharedObjects = SharedObjects.create(); + private static Configuration configuration() { + Configuration conf = new Configuration(); + + try { + FsStateChangelogStorageFactory.configure(conf, TEMPORARY_FOLDER.newFolder()); + } catch (IOException e) { + throw new RuntimeException(e); + } + + return conf; + } + @Parameter public TestingGraphBuilder graphBuilder; @Parameterized.Parameters(name = "{0}") @@ -68,7 +99,7 @@ public void test() throws Exception { .setCheckpointStorage( TEMPORARY_FOLDER.newFolder().toURI())); - TestJobExecutor.execute(testJob, MINI_CLUSTER_RESOURCE) + TestJobExecutor.execute(testJob, miniClusterResource) .waitForEvent(CheckpointCompletedEvent.class) .sendBroadcastCommand(FINISH_SOURCES, ALL_SUBTASKS) .waitForTermination() diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ChangelogPeriodicMaterializationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ChangelogPeriodicMaterializationITCase.java index f8e063b420b39..fb4f8752562c8 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ChangelogPeriodicMaterializationITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ChangelogPeriodicMaterializationITCase.java @@ -49,7 +49,7 @@ public void testNonMaterialization() throws Exception { SharedReference hasMaterialization = sharedObjects.add(new AtomicBoolean(true)); StreamExecutionEnvironment env = - getEnv(delegatedStateBackend, checkpointFolder, 1000, 1, Long.MAX_VALUE, 0); + getEnv(delegatedStateBackend, checkpointFolder, 1000, 1, -1, 0); waitAndAssert( buildJobGraph( env, diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ManualCheckpointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ManualCheckpointITCase.java index dc9c6647e828e..6e6250e47fbee 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ManualCheckpointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ManualCheckpointITCase.java @@ -24,7 +24,6 @@ import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.connector.source.mocks.MockSource; import org.apache.flink.configuration.Configuration; @@ -46,7 +45,6 @@ import org.junit.runners.Parameterized; import java.io.IOException; -import java.time.Duration; import java.util.Collections; import java.util.Optional; import java.util.concurrent.ExecutionException; @@ -91,10 +89,7 @@ public void testTriggeringWhenPeriodicDisabled() throws Exception { final JobID jobID = jobClient.getJobID(); final MiniCluster miniCluster = MINI_CLUSTER_RESOURCE.getMiniCluster(); - CommonTestUtils.waitForJobStatus( - jobClient, - Collections.singletonList(JobStatus.RUNNING), - Deadline.fromNow(Duration.ofSeconds(30))); + CommonTestUtils.waitForJobStatus(jobClient, Collections.singletonList(JobStatus.RUNNING)); CommonTestUtils.waitForAllTaskRunning(miniCluster, jobID, false); // wait for the checkpoint to be taken @@ -126,14 +121,10 @@ public void testTriggeringWhenPeriodicEnabled() throws Exception { final JobID jobID = jobClient.getJobID(); final MiniCluster miniCluster = MINI_CLUSTER_RESOURCE.getMiniCluster(); - CommonTestUtils.waitForJobStatus( - jobClient, - Collections.singletonList(JobStatus.RUNNING), - Deadline.fromNow(Duration.ofSeconds(30))); + CommonTestUtils.waitForJobStatus(jobClient, Collections.singletonList(JobStatus.RUNNING)); CommonTestUtils.waitForAllTaskRunning(miniCluster, jobID, false); CommonTestUtils.waitUntilCondition( () -> queryCompletedCheckpoints(miniCluster, jobID) > 0L, - Deadline.fromNow(Duration.ofSeconds(30)), checkpointingInterval / 2); final long numberOfPeriodicCheckpoints = queryCompletedCheckpoints(miniCluster, jobID); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java index eeda1b08b3237..0ea14a8d36aa4 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java @@ -18,25 +18,25 @@ package org.apache.flink.test.checkpointing; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.eventtime.AscendingTimestampsWatermarks; import org.apache.flink.api.common.eventtime.TimestampAssigner; import org.apache.flink.api.common.eventtime.TimestampAssignerSupplier; import org.apache.flink.api.common.eventtime.WatermarkGenerator; import org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier; import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.changelog.fs.FsStateChangelogStorageFactory; -import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.contrib.streaming.state.RocksDBStateBackend; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.state.filesystem.FsStateBackend; +import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -51,18 +51,17 @@ import org.junit.ClassRule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import javax.annotation.Nullable; import java.io.File; import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.Optional; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutionException; -import java.util.stream.Stream; +import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForCheckpoint; +import static org.apache.flink.test.util.TestUtils.waitUntilJobCanceled; import static org.junit.Assert.assertNotNull; /** @@ -74,26 +73,42 @@ *

        This tests considers full and incremental checkpoints and was introduced to guard against * problems like FLINK-6964. */ +@RunWith(Parameterized.class) public class ResumeCheckpointManuallyITCase extends TestLogger { private static final int PARALLELISM = 2; private static final int NUM_TASK_MANAGERS = 2; private static final int SLOTS_PER_TASK_MANAGER = 2; + @Parameterized.Parameter public RestoreMode restoreMode; + + @Parameterized.Parameters(name = "RestoreMode = {0}") + public static Object[] parameters() { + return RestoreMode.values(); + } + @ClassRule public static TemporaryFolder temporaryFolder = new TemporaryFolder(); @Test public void testExternalizedIncrementalRocksDBCheckpointsStandalone() throws Exception { final File checkpointDir = temporaryFolder.newFolder(); testExternalizedCheckpoints( - checkpointDir, null, createRocksDBStateBackend(checkpointDir, true), false); + checkpointDir, + null, + createRocksDBStateBackend(checkpointDir, true), + false, + restoreMode); } @Test public void testExternalizedFullRocksDBCheckpointsStandalone() throws Exception { final File checkpointDir = temporaryFolder.newFolder(); testExternalizedCheckpoints( - checkpointDir, null, createRocksDBStateBackend(checkpointDir, false), false); + checkpointDir, + null, + createRocksDBStateBackend(checkpointDir, false), + false, + restoreMode); } @Test @@ -101,7 +116,11 @@ public void testExternalizedIncrementalRocksDBCheckpointsWithLocalRecoveryStanda throws Exception { final File checkpointDir = temporaryFolder.newFolder(); testExternalizedCheckpoints( - checkpointDir, null, createRocksDBStateBackend(checkpointDir, true), true); + checkpointDir, + null, + createRocksDBStateBackend(checkpointDir, true), + true, + restoreMode); } @Test @@ -109,20 +128,25 @@ public void testExternalizedFullRocksDBCheckpointsWithLocalRecoveryStandalone() throws Exception { final File checkpointDir = temporaryFolder.newFolder(); testExternalizedCheckpoints( - checkpointDir, null, createRocksDBStateBackend(checkpointDir, false), true); + checkpointDir, + null, + createRocksDBStateBackend(checkpointDir, false), + true, + restoreMode); } @Test public void testExternalizedFSCheckpointsStandalone() throws Exception { final File checkpointDir = temporaryFolder.newFolder(); testExternalizedCheckpoints( - checkpointDir, null, createFsStateBackend(checkpointDir), false); + checkpointDir, null, createFsStateBackend(checkpointDir), false, restoreMode); } @Test public void testExternalizedFSCheckpointsWithLocalRecoveryStandalone() throws Exception { final File checkpointDir = temporaryFolder.newFolder(); - testExternalizedCheckpoints(checkpointDir, null, createFsStateBackend(checkpointDir), true); + testExternalizedCheckpoints( + checkpointDir, null, createFsStateBackend(checkpointDir), true, restoreMode); } @Test @@ -135,7 +159,8 @@ public void testExternalizedIncrementalRocksDBCheckpointsZookeeper() throws Exce checkpointDir, zkServer.getConnectString(), createRocksDBStateBackend(checkpointDir, true), - false); + false, + restoreMode); } finally { zkServer.stop(); } @@ -151,7 +176,8 @@ public void testExternalizedFullRocksDBCheckpointsZookeeper() throws Exception { checkpointDir, zkServer.getConnectString(), createRocksDBStateBackend(checkpointDir, false), - false); + false, + restoreMode); } finally { zkServer.stop(); } @@ -168,7 +194,8 @@ public void testExternalizedIncrementalRocksDBCheckpointsWithLocalRecoveryZookee checkpointDir, zkServer.getConnectString(), createRocksDBStateBackend(checkpointDir, true), - true); + true, + restoreMode); } finally { zkServer.stop(); } @@ -185,7 +212,8 @@ public void testExternalizedFullRocksDBCheckpointsWithLocalRecoveryZookeeper() checkpointDir, zkServer.getConnectString(), createRocksDBStateBackend(checkpointDir, false), - true); + true, + restoreMode); } finally { zkServer.stop(); } @@ -201,7 +229,8 @@ public void testExternalizedFSCheckpointsZookeeper() throws Exception { checkpointDir, zkServer.getConnectString(), createFsStateBackend(checkpointDir), - false); + false, + restoreMode); } finally { zkServer.stop(); } @@ -217,7 +246,8 @@ public void testExternalizedFSCheckpointsWithLocalRecoveryZookeeper() throws Exc checkpointDir, zkServer.getConnectString(), createFsStateBackend(checkpointDir), - true); + true, + restoreMode); } finally { zkServer.stop(); } @@ -233,8 +263,12 @@ private RocksDBStateBackend createRocksDBStateBackend( return new RocksDBStateBackend(checkpointDir.toURI().toString(), incrementalCheckpointing); } - private void testExternalizedCheckpoints( - File checkpointDir, String zooKeeperQuorum, StateBackend backend, boolean localRecovery) + private static void testExternalizedCheckpoints( + File checkpointDir, + String zooKeeperQuorum, + StateBackend backend, + boolean localRecovery, + RestoreMode restoreMode) throws Exception { final Configuration config = new Configuration(); @@ -270,22 +304,28 @@ private void testExternalizedCheckpoints( cluster.before(); - ClusterClient client = cluster.getClusterClient(); - try { // main test sequence: start job -> eCP -> restore job -> eCP -> restore job String firstExternalCheckpoint = - runJobAndGetExternalizedCheckpoint(backend, checkpointDir, null, client); + runJobAndGetExternalizedCheckpoint(backend, null, cluster, restoreMode); assertNotNull(firstExternalCheckpoint); String secondExternalCheckpoint = runJobAndGetExternalizedCheckpoint( - backend, checkpointDir, firstExternalCheckpoint, client); + backend, firstExternalCheckpoint, cluster, restoreMode); assertNotNull(secondExternalCheckpoint); String thirdExternalCheckpoint = runJobAndGetExternalizedCheckpoint( - backend, checkpointDir, secondExternalCheckpoint, client); + backend, + // in CLAIM mode, the previous run is only guaranteed to preserve the + // latest checkpoint; in NO_CLAIM/LEGACY, even the initial checkpoints + // must remain valid + restoreMode == RestoreMode.CLAIM + ? secondExternalCheckpoint + : firstExternalCheckpoint, + cluster, + restoreMode); assertNotNull(thirdExternalCheckpoint); } finally { cluster.after(); @@ -294,77 +334,31 @@ private void testExternalizedCheckpoints( private static String runJobAndGetExternalizedCheckpoint( StateBackend backend, - File checkpointDir, @Nullable String externalCheckpoint, - ClusterClient client) + MiniClusterWithClientResource cluster, + RestoreMode restoreMode) throws Exception { - JobGraph initialJobGraph = getJobGraph(backend, externalCheckpoint); + JobGraph initialJobGraph = getJobGraph(backend, externalCheckpoint, restoreMode); NotifyingInfiniteTupleSource.countDownLatch = new CountDownLatch(PARALLELISM); - - client.submitJob(initialJobGraph).get(); + cluster.getClusterClient().submitJob(initialJobGraph).get(); // wait until all sources have been started NotifyingInfiniteTupleSource.countDownLatch.await(); - waitUntilExternalizedCheckpointCreated(checkpointDir, initialJobGraph.getJobID()); - client.cancel(initialJobGraph.getJobID()).get(); - waitUntilCanceled(initialJobGraph.getJobID(), client); - - return getExternalizedCheckpointCheckpointPath(checkpointDir, initialJobGraph.getJobID()); - } - - private static String getExternalizedCheckpointCheckpointPath(File checkpointDir, JobID jobId) - throws IOException { - Optional checkpoint = findExternalizedCheckpoint(checkpointDir, jobId); - if (!checkpoint.isPresent()) { - throw new AssertionError("No complete checkpoint could be found."); - } else { - return checkpoint.get().toString(); - } - } - - private static void waitUntilExternalizedCheckpointCreated(File checkpointDir, JobID jobId) - throws InterruptedException, IOException { - while (true) { - Thread.sleep(50); - Optional externalizedCheckpoint = - findExternalizedCheckpoint(checkpointDir, jobId); - if (externalizedCheckpoint.isPresent()) { - break; - } - } - } - - private static Optional findExternalizedCheckpoint(File checkpointDir, JobID jobId) - throws IOException { - try (Stream checkpoints = - Files.list(checkpointDir.toPath().resolve(jobId.toString()))) { - return checkpoints - .filter(path -> path.getFileName().toString().startsWith("chk-")) - .filter( - path -> { - try (Stream checkpointFiles = Files.list(path)) { - return checkpointFiles.anyMatch( - child -> - child.getFileName() - .toString() - .contains("meta")); - } catch (IOException ignored) { - return false; - } - }) - .findAny(); - } - } - - private static void waitUntilCanceled(JobID jobId, ClusterClient client) - throws ExecutionException, InterruptedException { - while (client.getJobStatus(jobId).get() != JobStatus.CANCELED) { - Thread.sleep(50); - } + // complete at least two checkpoints so that the initial checkpoint can be subsumed + waitForCheckpoint(initialJobGraph.getJobID(), cluster.getMiniCluster(), 2); + cluster.getClusterClient().cancel(initialJobGraph.getJobID()).get(); + waitUntilJobCanceled(initialJobGraph.getJobID(), cluster.getClusterClient()); + return CommonTestUtils.getLatestCompletedCheckpointPath( + initialJobGraph.getJobID(), cluster.getMiniCluster()) + .orElseThrow( + () -> { + throw new IllegalStateException("Checkpoint not generated"); + }); } - private static JobGraph getJobGraph(StateBackend backend, @Nullable String externalCheckpoint) { + private static JobGraph getJobGraph( + StateBackend backend, @Nullable String externalCheckpoint, RestoreMode restoreMode) { final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.enableCheckpointing(500); @@ -373,6 +367,7 @@ private static JobGraph getJobGraph(StateBackend backend, @Nullable String exter env.getCheckpointConfig() .setExternalizedCheckpointCleanup( CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION); + env.setRestartStrategy(RestartStrategies.noRestart()); env.addSource(new NotifyingInfiniteTupleSource(10_000)) .assignTimestampsAndWatermarks(IngestionTimeWatermarkStrategy.create()) @@ -388,7 +383,7 @@ private static JobGraph getJobGraph(StateBackend backend, @Nullable String exter // recover from previous iteration? if (externalCheckpoint != null) { jobGraph.setSavepointRestoreSettings( - SavepointRestoreSettings.forPath(externalCheckpoint)); + SavepointRestoreSettings.forPath(externalCheckpoint, false, restoreMode)); } return jobGraph; diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java index acc0ddd8a1eef..b41febb04ee09 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java @@ -59,9 +59,11 @@ import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.runtime.messages.FlinkJobNotFoundException; +import org.apache.flink.runtime.operators.testutils.ExpectedTestException; import org.apache.flink.runtime.rest.messages.EmptyRequestBody; import org.apache.flink.runtime.rest.messages.JobMessageParameters; import org.apache.flink.runtime.rest.messages.job.JobDetailsHeaders; +import org.apache.flink.runtime.scheduler.stopwithsavepoint.StopWithSavepointStoppingException; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.runtime.state.StateSnapshotContext; @@ -143,10 +145,13 @@ import static org.apache.flink.util.ExceptionUtils.assertThrowableWithMessage; import static org.apache.flink.util.ExceptionUtils.findThrowable; import static org.apache.flink.util.ExceptionUtils.findThrowableWithMessage; +import static org.hamcrest.CoreMatchers.either; +import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -292,6 +297,79 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { public void initializeState(FunctionInitializationContext context) throws Exception {} } + @Test + public void testStopWithSavepointFailsOverToSavepoint() throws Throwable { + int sinkParallelism = 5; + MiniClusterWithClientResource cluster = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberSlotsPerTaskManager(sinkParallelism + 1) + .build()); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.getConfig().setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 10)); + env.setParallelism(1); + env.addSource(new InfiniteTestSource()) + .name("Infinite Source") + .map(new FailingOnCompletedSavepointMapFunction(2)) + .addSink(new DiscardingSink<>()) + // different parallelism to break chaining and add some concurrent tasks + .setParallelism(sinkParallelism); + + final JobGraph jobGraph = env.getStreamGraph().getJobGraph(); + + cluster.before(); + try { + ClusterClient client = cluster.getClusterClient(); + client.submitJob(jobGraph).get(); + waitUntilAllTasksAreRunning(cluster.getRestClusterClient(), jobGraph.getJobID()); + + cluster.getMiniCluster().triggerCheckpoint(jobGraph.getJobID()).get(); + final CompletableFuture savepointCompleted = + client.stopWithSavepoint( + jobGraph.getJobID(), + true, + savepointDir.getAbsolutePath(), + SavepointFormatType.CANONICAL); + + final Throwable savepointException = + assertThrows(ExecutionException.class, savepointCompleted::get).getCause(); + assertThrowable( + savepointException, + throwable -> + throwable instanceof StopWithSavepointStoppingException + && throwable + .getMessage() + .startsWith("A savepoint has been created at: ")); + assertThat( + client.getJobStatus(jobGraph.getJobID()).get(), + either(is(JobStatus.FAILED)).or(is(JobStatus.FAILING))); + } finally { + cluster.after(); + } + } + + private static final class FailingOnCompletedSavepointMapFunction + extends RichMapFunction implements CheckpointListener { + private final long savepointId; + + private FailingOnCompletedSavepointMapFunction(long savepointId) { + this.savepointId = savepointId; + } + + @Override + public Integer map(Integer value) throws Exception { + return value; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + if (checkpointId == savepointId) { + throw new ExpectedTestException(); + } + } + } + /** * Triggers a savepoint for a job that uses the FsStateBackend. We expect that all checkpoint * files are written to a new savepoint directory. @@ -924,7 +1002,8 @@ public void testStopWithSavepointFailingInSnapshotCreation() throws Exception { // 1. task failure restart // 2. job failover triggered by the CheckpointFailureManager 2, - assertInSnapshotCreationFailure()); + assertInSnapshotCreationFailure(), + true); } @Test @@ -937,8 +1016,16 @@ public void testStopWithSavepointFailingAfterSnapshotCreation() throws Exception // two restarts expected: // 1. task failure restart // 2. job failover triggered by SchedulerBase.stopWithSavepoint - 2, - assertAfterSnapshotCreationFailure()); + 0, + (jobId, actualException) -> { + Optional actualFlinkException = + findThrowable( + actualException, StopWithSavepointStoppingException.class); + return actualFlinkException + .map(e -> e.getMessage().startsWith("A savepoint has been created at:")) + .orElse(false); + }, + false); } @Test @@ -991,30 +1078,6 @@ public void testStopWithSavepointWithDrainGlobalFailoverIfSavepointAborted() thr } } - private static BiFunction - assertAfterSnapshotCreationFailure() { - return (jobId, actualException) -> { - if (ClusterOptions.isAdaptiveSchedulerEnabled(new Configuration())) { - return actualException - .getMessage() - .contains("Stop with savepoint operation could not be completed"); - } else { - Optional actualFlinkException = - findThrowable(actualException, FlinkException.class); - if (!actualFlinkException.isPresent()) { - return false; - } - return actualFlinkException - .get() - .getMessage() - .contains( - String.format( - "A global fail-over is triggered to recover the job %s.", - jobId)); - } - }; - } - private static BiFunction assertInSnapshotCreationFailure() { return (ignored, actualException) -> { @@ -1051,7 +1114,8 @@ private static void testStopWithFailingSourceInOnePipeline( InfiniteTestSource failingSource, File savepointDir, int expectedMaximumNumberOfRestarts, - BiFunction exceptionAssertion) + BiFunction exceptionAssertion, + boolean shouldRestart) throws Exception { MiniClusterWithClientResource cluster = new MiniClusterWithClientResource( @@ -1107,7 +1171,9 @@ private static void testStopWithFailingSourceInOnePipeline( assertThrowable(e, ex -> exceptionAssertion.apply(jobGraph.getJobID(), e)); } - waitUntilAllTasksAreRunning(cluster.getRestClusterClient(), jobGraph.getJobID()); + if (shouldRestart) { + waitUntilAllTasksAreRunning(cluster.getRestClusterClient(), jobGraph.getJobID()); + } } finally { cluster.after(); } @@ -1130,8 +1196,7 @@ public static void waitUntilAllTasksAreRunning( detailsInfo -> allVerticesRunning( detailsInfo.getJobVerticesPerState())) - .get(), - Deadline.fromNow(Duration.ofSeconds(10))); + .get()); } private static boolean allVerticesRunning(Map states) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UnalignedCheckpointFailureHandlingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UnalignedCheckpointFailureHandlingITCase.java index e834ff9229cf9..dac1536e434e1 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UnalignedCheckpointFailureHandlingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UnalignedCheckpointFailureHandlingITCase.java @@ -63,7 +63,6 @@ import static java.util.Collections.singletonList; import static org.apache.flink.api.common.JobStatus.RUNNING; -import static org.apache.flink.api.common.time.Deadline.fromNow; import static org.apache.flink.core.fs.Path.fromLocalFile; import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForAllTaskRunning; import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForJobStatus; @@ -105,7 +104,7 @@ public void testCheckpointSuccessAfterFailure() throws Exception { JobID jobID = jobClient.getJobID(); MiniCluster miniCluster = miniClusterResource.getMiniCluster(); - waitForJobStatus(jobClient, singletonList(RUNNING), fromNow(Duration.ofSeconds(30))); + waitForJobStatus(jobClient, singletonList(RUNNING)); waitForAllTaskRunning(miniCluster, jobID, false); triggerFailingCheckpoint(jobID, TestException.class, miniCluster); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ClusterEntrypointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ClusterEntrypointITCase.java index c1b6ccccaa6e9..186f531736f3f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ClusterEntrypointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ClusterEntrypointITCase.java @@ -18,7 +18,6 @@ package org.apache.flink.test.recovery; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.ClusterOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; @@ -38,7 +37,6 @@ import java.io.File; import java.nio.file.Files; import java.nio.file.Path; -import java.time.Duration; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -73,8 +71,7 @@ public void testDeterministicWorkingDirectoryIsNotDeletedInCaseOfProcessFailure( boolean success = false; try { - CommonTestUtils.waitUntilCondition( - workingDirectory::exists, Deadline.fromNow(Duration.ofMinutes(1L))); + CommonTestUtils.waitUntilCondition(workingDirectory::exists); jobManagerProcess.getProcess().destroy(); @@ -112,8 +109,7 @@ public void testNondeterministicWorkingDirectoryIsDeletedInCaseOfProcessFailure( try (Stream files = Files.list(workingDirBase.toPath())) { return files.findAny().isPresent(); } - }, - Deadline.fromNow(Duration.ofMinutes(1L))); + }); final File workingDirectory = Iterables.getOnlyElement( diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java index 00c977f14ef1a..c41001f674220 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java @@ -309,7 +309,7 @@ public void testDispatcherProcessFailure() throws Exception { leaderRetrievalService.start(leaderListener); // Initial submission - leaderListener.waitForNewLeader(deadline.timeLeft().toMillis()); + leaderListener.waitForNewLeader(); String leaderAddress = leaderListener.getAddress(); UUID leaderId = leaderListener.getLeaderSessionID(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/LocalRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/LocalRecoveryITCase.java index 108cf2f9085f4..1639cda95b92a 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/LocalRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/LocalRecoveryITCase.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.ClusterOptions; import org.apache.flink.configuration.Configuration; @@ -105,10 +104,7 @@ public void testRecoverLocallyFromProcessCrashWithWorkingDirectory() throws Exce final long waitingTimeInSeconds = 45L; waitUntilCheckpointCompleted( - configuration, - clusterEntrypoint.getRestPort(), - jobClient.getJobID(), - Deadline.fromNow(Duration.ofSeconds(waitingTimeInSeconds))); + configuration, clusterEntrypoint.getRestPort(), jobClient.getJobID()); restartTaskManagerProcesses(taskManagerProcesses, parallelism - 1); @@ -219,8 +215,7 @@ public void terminate() { } private void waitUntilCheckpointCompleted( - Configuration configuration, int restPort, JobID jobId, Deadline deadline) - throws Exception { + Configuration configuration, int restPort, JobID jobId) throws Exception { final RestClient restClient = new RestClient(configuration, Executors.directExecutor()); final JobMessageParameters messageParameters = new JobMessageParameters(); messageParameters.jobPathParameter.resolve(jobId); @@ -237,8 +232,7 @@ private void waitUntilCheckpointCompleted( EmptyRequestBody.getInstance()) .join(); return checkpointingStatistics.getCounts().getNumberCompletedCheckpoints() > 0; - }, - deadline); + }); } private JobClient submitJob( diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerRunnerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerRunnerITCase.java index cd7824dd85391..64b2e7c1b2a75 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerRunnerITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerRunnerITCase.java @@ -18,7 +18,6 @@ package org.apache.flink.test.recovery; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.ClusterOptions; import org.apache.flink.configuration.Configuration; @@ -76,8 +75,7 @@ public void testDeterministicWorkingDirIsNotDeletedInCaseOfProcessFailure() thro boolean success = false; try { - CommonTestUtils.waitUntilCondition( - workingDirectory::exists, Deadline.fromNow(Duration.ofMinutes(1L))); + CommonTestUtils.waitUntilCondition(workingDirectory::exists); taskManagerProcess.getProcess().destroy(); @@ -115,8 +113,7 @@ public void testNondeterministicWorkingDirIsDeletedInCaseOfProcessFailure() thro try (Stream files = Files.list(workingDirBase.toPath())) { return files.findAny().isPresent(); } - }, - Deadline.fromNow(Duration.ofMinutes(1L))); + }); final File workingDirectory = Iterables.getOnlyElement( diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/DefaultSchedulerLocalRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/DefaultSchedulerLocalRecoveryITCase.java index 2f16fe75a5882..0bcccecaaa80c 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/DefaultSchedulerLocalRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/DefaultSchedulerLocalRecoveryITCase.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.client.program.MiniClusterClient; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; @@ -47,7 +46,6 @@ import org.junit.experimental.categories.Category; import java.io.IOException; -import java.time.Duration; import java.util.Arrays; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -160,9 +158,7 @@ private ArchivedExecutionGraph executeSchedulingTest( private void waitUntilAllVerticesRunning(JobID jobId, MiniCluster miniCluster) throws Exception { CommonTestUtils.waitForAllTaskRunning( - () -> miniCluster.getExecutionGraph(jobId).get(TIMEOUT, TimeUnit.SECONDS), - Deadline.fromNow(Duration.ofMillis(TIMEOUT)), - false); + () -> miniCluster.getExecutionGraph(jobId).get(TIMEOUT, TimeUnit.SECONDS), false); } private JobGraph createJobGraph(int parallelism) throws IOException { diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java index 8aab37a88e251..2ebd53564e419 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.ClusterOptions; import org.apache.flink.configuration.Configuration; @@ -63,8 +62,6 @@ /** Test the election of a new JobManager leader. */ public class ZooKeeperLeaderElectionITCase extends TestLogger { - private static final Duration TEST_TIMEOUT = Duration.ofMinutes(5L); - private static final Time RPC_TIMEOUT = Time.minutes(1L); private static TestingServer zkServer; @@ -111,8 +108,6 @@ public void testJobExecutionOnClusterWithLeaderChange() throws Exception { .setNumSlotsPerTaskManager(numSlotsPerTM) .build(); - final Deadline timeout = Deadline.fromNow(TEST_TIMEOUT); - try (TestingMiniCluster miniCluster = TestingMiniCluster.newBuilder(miniClusterConfiguration).build(); final CuratorFrameworkWithUnhandledErrorListener curatorFramework = @@ -146,20 +141,19 @@ public void testJobExecutionOnClusterWithLeaderChange() throws Exception { for (int i = 0; i < numDispatchers - 1; i++) { final DispatcherGateway leaderDispatcherGateway = - getNextLeadingDispatcherGateway( - miniCluster, previousLeaderAddress, timeout); + getNextLeadingDispatcherGateway(miniCluster, previousLeaderAddress); // Make sure resource manager has also changed leadership. resourceManagerLeaderFutures[i].get(); previousLeaderAddress = leaderDispatcherGateway.getAddress(); - awaitRunningStatus(leaderDispatcherGateway, jobGraph, timeout); + awaitRunningStatus(leaderDispatcherGateway, jobGraph); leaderDispatcherGateway.shutDownCluster(); } final DispatcherGateway leaderDispatcherGateway = - getNextLeadingDispatcherGateway(miniCluster, previousLeaderAddress, timeout); + getNextLeadingDispatcherGateway(miniCluster, previousLeaderAddress); // Make sure resource manager has also changed leadership. resourceManagerLeaderFutures[numDispatchers - 1].get(); - awaitRunningStatus(leaderDispatcherGateway, jobGraph, timeout); + awaitRunningStatus(leaderDispatcherGateway, jobGraph); CompletableFuture jobResultFuture = leaderDispatcherGateway.requestJobResult(jobGraph.getJobID(), RPC_TIMEOUT); BlockingOperator.unblock(); @@ -170,21 +164,17 @@ public void testJobExecutionOnClusterWithLeaderChange() throws Exception { } } - private static void awaitRunningStatus( - DispatcherGateway dispatcherGateway, JobGraph jobGraph, Deadline timeout) + private static void awaitRunningStatus(DispatcherGateway dispatcherGateway, JobGraph jobGraph) throws Exception { CommonTestUtils.waitUntilCondition( () -> dispatcherGateway.requestJobStatus(jobGraph.getJobID(), RPC_TIMEOUT).get() == JobStatus.RUNNING, - timeout, 50L); } private DispatcherGateway getNextLeadingDispatcherGateway( - TestingMiniCluster miniCluster, - @Nullable String previousLeaderAddress, - Deadline timeout) + TestingMiniCluster miniCluster, @Nullable String previousLeaderAddress) throws Exception { CommonTestUtils.waitUntilCondition( () -> @@ -193,7 +183,6 @@ private DispatcherGateway getNextLeadingDispatcherGateway( .get() .getAddress() .equals(previousLeaderAddress), - timeout, 20L); return miniCluster.getDispatcherGatewayFuture().get(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/scheduling/AdaptiveSchedulerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/scheduling/AdaptiveSchedulerITCase.java index 3852bd19c982d..a4124dfe08c34 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/scheduling/AdaptiveSchedulerITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/scheduling/AdaptiveSchedulerITCase.java @@ -23,7 +23,6 @@ import org.apache.flink.api.common.state.CheckpointListener; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.ClusterOptions; import org.apache.flink.configuration.Configuration; @@ -35,6 +34,7 @@ import org.apache.flink.runtime.rest.messages.JobExceptionsHeaders; import org.apache.flink.runtime.rest.messages.JobExceptionsInfoWithHistory; import org.apache.flink.runtime.rest.messages.job.JobExceptionsMessageParameters; +import org.apache.flink.runtime.scheduler.stopwithsavepoint.StopWithSavepointStoppingException; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.runtime.testutils.CommonTestUtils; @@ -60,8 +60,6 @@ import javax.annotation.Nullable; import java.io.File; -import java.time.Duration; -import java.time.temporal.ChronoUnit; import java.util.Arrays; import java.util.Collections; import java.util.concurrent.CompletableFuture; @@ -70,9 +68,12 @@ import java.util.stream.Collectors; import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.util.ExceptionUtils.assertThrowable; import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.CoreMatchers.either; import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.fail; import static org.junit.Assume.assumeTrue; @@ -179,13 +180,11 @@ public void testStopWithSavepointFailOnCheckpoint() throws Exception { assertThat(e, containsCause(FlinkException.class)); } // expect job to run again (maybe restart) - CommonTestUtils.waitUntilCondition( - () -> client.getJobStatus().get() == JobStatus.RUNNING, - Deadline.fromNow(Duration.of(1, ChronoUnit.MINUTES))); + CommonTestUtils.waitUntilCondition(() -> client.getJobStatus().get() == JobStatus.RUNNING); } @Test - public void testStopWithSavepointFailOnStop() throws Exception { + public void testStopWithSavepointFailOnStop() throws Throwable { StreamExecutionEnvironment env = getEnvWithSource(StopWithSavepointTestBehavior.FAIL_ON_CHECKPOINT_COMPLETE); env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 0L)); @@ -195,20 +194,23 @@ public void testStopWithSavepointFailOnStop() throws Exception { JobClient client = env.executeAsync(); DummySource.awaitRunning(); - try { - client.stopWithSavepoint( - false, - tempFolder.newFolder("savepoint").getAbsolutePath(), - SavepointFormatType.CANONICAL) - .get(); - fail("Expect exception"); - } catch (ExecutionException e) { - assertThat(e, containsCause(FlinkException.class)); - } - // expect job to run again (maybe restart) - CommonTestUtils.waitUntilCondition( - () -> client.getJobStatus().get() == JobStatus.RUNNING, - Deadline.fromNow(Duration.of(1, ChronoUnit.MINUTES))); + final CompletableFuture savepointCompleted = + client.stopWithSavepoint( + false, + tempFolder.newFolder("savepoint").getAbsolutePath(), + SavepointFormatType.CANONICAL); + final Throwable savepointException = + assertThrows(ExecutionException.class, savepointCompleted::get).getCause(); + assertThrowable( + savepointException, + throwable -> + throwable instanceof StopWithSavepointStoppingException + && throwable + .getMessage() + .startsWith("A savepoint has been created at: ")); + assertThat( + client.getJobStatus().get(), + either(is(JobStatus.FAILED)).or(is(JobStatus.FAILING))); } @Test @@ -243,9 +245,7 @@ public void testStopWithSavepointFailOnFirstSavepointSucceedOnSecond() throws Ex // ensure failed savepoint files have been removed from the directory. // We execute this in a retry loop with a timeout, because the savepoint deletion happens // asynchronously and is not bound to the job lifecycle. See FLINK-22493 for more details. - CommonTestUtils.waitUntilCondition( - () -> isDirectoryEmpty(savepointDirectory), - Deadline.fromNow(Duration.ofSeconds(10))); + CommonTestUtils.waitUntilCondition(() -> isDirectoryEmpty(savepointDirectory)); // trigger second savepoint final String savepoint = @@ -259,7 +259,6 @@ public void testStopWithSavepointFailOnFirstSavepointSucceedOnSecond() throws Ex @Test public void testExceptionHistoryIsRetrievableFromTheRestAPI() throws Exception { - final Deadline deadline = Deadline.fromNow(Duration.ofMinutes(1)); final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(PARALLELISM); env.enableCheckpointing(20L, CheckpointingMode.EXACTLY_ONCE); @@ -281,11 +280,9 @@ public void testExceptionHistoryIsRetrievableFromTheRestAPI() throws Exception { exceptionsFuture.get(); return jobExceptionsInfoWithHistory.getExceptionHistory().getEntries().size() > 0; - }, - deadline); + }); jobClient.cancel().get(); - CommonTestUtils.waitForJobStatus( - jobClient, Collections.singletonList(JobStatus.CANCELED), deadline); + CommonTestUtils.waitForJobStatus(jobClient, Collections.singletonList(JobStatus.CANCELED)); } private boolean isDirectoryEmpty(File directory) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/scheduling/ReactiveModeITCase.java b/flink-tests/src/test/java/org/apache/flink/test/scheduling/ReactiveModeITCase.java index 9859a87a7bf83..ee09c293b8ac5 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/scheduling/ReactiveModeITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/scheduling/ReactiveModeITCase.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; @@ -40,7 +39,6 @@ import org.junit.Rule; import org.junit.Test; -import java.time.Duration; import java.util.concurrent.ExecutionException; /** Tests for Reactive Mode (FLIP-159). */ @@ -146,9 +144,7 @@ private int getNumberOfConnectedTaskManagers() throws ExecutionException, Interr private void startAdditionalTaskManager() throws Exception { miniClusterResource.getMiniCluster().startTaskManager(); - CommonTestUtils.waitUntilCondition( - () -> getNumberOfConnectedTaskManagers() == 2, - Deadline.fromNow(Duration.ofMillis(10_000L))); + CommonTestUtils.waitUntilCondition(() -> getNumberOfConnectedTaskManagers() == 2); } private static class DummySource implements SourceFunction { @@ -213,7 +209,6 @@ public static void waitUntilParallelismForVertexReached( } } return false; - }, - Deadline.fromNow(Duration.ofSeconds(10))); + }); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogCompatibilityITCase.java b/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogCompatibilityITCase.java index 3023e996271b2..aafa09668789b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogCompatibilityITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogCompatibilityITCase.java @@ -23,6 +23,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.SavepointFormatType; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.DiscardingSink; @@ -39,14 +40,15 @@ import java.io.File; import java.util.Arrays; import java.util.List; +import java.util.NoSuchElementException; import java.util.Optional; import static org.apache.flink.configuration.CheckpointingOptions.CHECKPOINTS_DIRECTORY; import static org.apache.flink.configuration.CheckpointingOptions.SAVEPOINT_DIRECTORY; import static org.apache.flink.runtime.jobgraph.SavepointRestoreSettings.forPath; import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForAllTaskRunning; +import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForCheckpoint; import static org.apache.flink.streaming.api.environment.CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION; -import static org.apache.flink.test.util.TestUtils.getMostRecentCompletedCheckpointMaybe; import static org.apache.flink.util.ExceptionUtils.findThrowableSerializedAware; import static org.junit.Assert.fail; @@ -141,12 +143,15 @@ private String tryCheckpointAndStop(JobGraph jobGraph) throws Exception { ClusterClient client = miniClusterResource.getClusterClient(); submit(jobGraph, client); if (testCase.restoreSource == RestoreSource.CHECKPOINT) { - while (!getMostRecentCompletedCheckpointMaybe(checkpointDir).isPresent()) { - Thread.sleep(50); - } + waitForCheckpoint(jobGraph.getJobID(), miniClusterResource.getMiniCluster(), 1); client.cancel(jobGraph.getJobID()).get(); // obtain the latest checkpoint *after* cancellation - that one won't be subsumed - return pathToString(getMostRecentCompletedCheckpointMaybe(checkpointDir).get()); + return CommonTestUtils.getLatestCompletedCheckpointPath( + jobGraph.getJobID(), miniClusterResource.getMiniCluster()) + .orElseThrow( + () -> { + throw new NoSuchElementException("No checkpoint was created yet"); + }); } else { return client.stopWithSavepoint( jobGraph.getJobID(), diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogRescalingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogRescalingITCase.java new file mode 100644 index 0000000000000..ed7b3d9e28923 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogRescalingITCase.java @@ -0,0 +1,351 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.test.state; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.lib.NumberSequenceSource; +import org.apache.flink.api.connector.source.lib.util.IteratorSourceReader; +import org.apache.flink.api.connector.source.lib.util.IteratorSourceSplit; +import org.apache.flink.changelog.fs.FsStateChangelogStorageFactory; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.core.io.InputStatus; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.CheckpointingMode; +import org.apache.flink.streaming.api.datastream.DataStreamUtils; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.DiscardingSink; +import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction; +import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.Collector; +import org.apache.flink.util.TestLogger; + +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +import java.io.File; +import java.io.Serializable; +import java.time.Duration; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.Timer; +import java.util.TimerTask; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Consumer; + +import static org.apache.flink.changelog.fs.FsStateChangelogOptions.PREEMPTIVE_PERSIST_THRESHOLD; +import static org.apache.flink.configuration.CheckpointingOptions.CHECKPOINTS_DIRECTORY; +import static org.apache.flink.configuration.CheckpointingOptions.CHECKPOINT_STORAGE; +import static org.apache.flink.configuration.CheckpointingOptions.FS_SMALL_FILE_THRESHOLD; +import static org.apache.flink.configuration.CheckpointingOptions.LOCAL_RECOVERY; +import static org.apache.flink.configuration.CoreOptions.DEFAULT_PARALLELISM; +import static org.apache.flink.configuration.PipelineOptions.OBJECT_REUSE; +import static org.apache.flink.configuration.RestartStrategyOptions.RESTART_STRATEGY; +import static org.apache.flink.configuration.StateBackendOptions.STATE_BACKEND; +import static org.apache.flink.configuration.StateChangelogOptions.ENABLE_STATE_CHANGE_LOG; +import static org.apache.flink.configuration.StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL; +import static org.apache.flink.configuration.TaskManagerOptions.BUFFER_DEBLOAT_ENABLED; +import static org.apache.flink.runtime.jobgraph.SavepointRestoreSettings.forPath; +import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForAllTaskRunning; +import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForCheckpoint; +import static org.apache.flink.streaming.api.environment.CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION; +import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.ALIGNED_CHECKPOINT_TIMEOUT; +import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL; +import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_MODE; +import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.ENABLE_UNALIGNED; +import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.EXTERNALIZED_CHECKPOINT; +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * Tests rescaling with Changelog enabled and with timers in state. It uses an adaptation of a + * ChangelogTestProgram that aims to generate the highest load possible while still allowing + * checkpointing. For that, it uses rate-limited FLIP-27 source and Unaligned checkpoints. + */ +@RunWith(Parameterized.class) +public class ChangelogRescalingITCase extends TestLogger { + /** The rate at which events will be generated by the source. */ + private static final int EVENTS_PER_SECOND_PER_READER = 100; + /** Payload size of each event generated randomly. */ + private static final int PAYLOAD_SIZE = 1000; + /** Size of (ProcessingTime) windows. */ + private static final Time WINDOW_SIZE = Time.milliseconds(100); + /** Slide size. */ + private static final Time WINDOW_SLIDE = Time.milliseconds(10); + /** Time to Accumulate some timer delete operations. */ + private static final int ACCUMULATE_TIME_MILLIS = 5_000; + + @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @Parameters(name = "Rescale {0} -> {1}") + public static Object[] parameters() { + return new Object[][] {new Object[] {6, 4}, new Object[] {4, 6}}; + } + + private final int parallelism1; + private final int parallelism2; + + private MiniClusterWithClientResource cluster; + + public ChangelogRescalingITCase(int parallelism1, int parallelism2) { + this.parallelism1 = parallelism1; + this.parallelism2 = parallelism2; + } + + @Before + public void before() throws Exception { + Configuration configuration = new Configuration(); + FsStateChangelogStorageFactory.configure(configuration, temporaryFolder.newFolder()); + cluster = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setConfiguration(configuration) + .setNumberSlotsPerTaskManager(Math.max(parallelism1, parallelism2)) + .build()); + cluster.before(); + } + + @After + public void after() { + if (cluster != null) { + cluster.after(); + cluster = null; + } + } + + @Test + public void test() throws Exception { + // before rescale + JobID jobID1 = submit(configureJob(parallelism1, temporaryFolder.newFolder()), graph -> {}); + + Thread.sleep(ACCUMULATE_TIME_MILLIS); + String cpLocation = checkpointAndCancel(jobID1); + + // rescale and checkpoint to verify + JobID jobID2 = + submit( + configureJob(parallelism2, temporaryFolder.newFolder()), + graph -> graph.setSavepointRestoreSettings(forPath(cpLocation))); + waitForAllTaskRunning(cluster.getMiniCluster(), jobID2, true); + cluster.getClusterClient().cancel(jobID2).get(); + } + + private JobID submit(Configuration conf, Consumer updateGraph) + throws InterruptedException, ExecutionException { + JobGraph jobGraph = createJobGraph(conf); + updateGraph.accept(jobGraph); + return cluster.getClusterClient().submitJob(jobGraph).get(); + } + + private JobGraph createJobGraph(Configuration conf) { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(conf); + SingleOutputStreamOperator map = + env.fromSource( + new ThrottlingNumberSequenceSource( + 0, Long.MAX_VALUE, EVENTS_PER_SECOND_PER_READER), + WatermarkStrategy.noWatermarks(), + "Sequence Source") + .keyBy(ChangelogRescalingITCase::key) + .map( + el -> { + // Thread.sleep(100); // don't block barriers + byte[] bytes = new byte[PAYLOAD_SIZE]; + ThreadLocalRandom.current().nextBytes(bytes); + return new TestEvent(el, bytes); + }); + DataStreamUtils.reinterpretAsKeyedStream(map, e -> key(e.id)) + .window(SlidingProcessingTimeWindows.of(WINDOW_SIZE, WINDOW_SLIDE)) + .process( + new ProcessWindowFunction() { + @Override + public void process( + Long key, + ProcessWindowFunction + .Context + context, + Iterable elements, + Collector out) {} + }) + .addSink(new DiscardingSink<>()); + + return env.getStreamGraph().getJobGraph(); + } + + private static long key(Long num) { + return num % 1000; + } + + private Configuration configureJob(int parallelism, File cpDir) { + Configuration conf = new Configuration(); + + conf.set(EXTERNALIZED_CHECKPOINT, RETAIN_ON_CANCELLATION); + conf.set(DEFAULT_PARALLELISM, parallelism); + conf.set(ENABLE_STATE_CHANGE_LOG, true); + conf.set(CHECKPOINTING_MODE, CheckpointingMode.EXACTLY_ONCE); + conf.set(CHECKPOINTING_INTERVAL, Duration.ofMillis(10)); + conf.set(CHECKPOINT_STORAGE, "filesystem"); + conf.set(CHECKPOINTS_DIRECTORY, cpDir.toURI().toString()); + conf.set(STATE_BACKEND, "hashmap"); + conf.set(LOCAL_RECOVERY, false); // not supported by changelog + // tune changelog + conf.set(PREEMPTIVE_PERSIST_THRESHOLD, MemorySize.ofMebiBytes(10)); + conf.set(PERIODIC_MATERIALIZATION_INTERVAL, Duration.ofMinutes(3)); + // tune flink + conf.set(FS_SMALL_FILE_THRESHOLD, MemorySize.ofMebiBytes(1)); + conf.set(OBJECT_REUSE, true); + + conf.set(ENABLE_UNALIGNED, true); // speedup + conf.set(ALIGNED_CHECKPOINT_TIMEOUT, Duration.ZERO); // prevent randomization + conf.set(BUFFER_DEBLOAT_ENABLED, false); // prevent randomization + conf.set(RESTART_STRATEGY, "none"); // not expecting any failures + + return conf; + } + + private static final class TestEvent implements Serializable { + private final long id; + + @SuppressWarnings({"FieldCanBeLocal", "unused"}) + private final byte[] payload; + + private TestEvent(long id, byte[] payload) { + this.id = id; + this.payload = payload; + } + } + + private static class ThrottlingNumberSequenceSource extends NumberSequenceSource { + private final int numbersPerSecond; + + public ThrottlingNumberSequenceSource(long from, long to, int numbersPerSecondPerReader) { + super(from, to); + this.numbersPerSecond = numbersPerSecondPerReader; + } + + @Override + public SourceReader createReader( + SourceReaderContext readerContext) { + return new ThrottlingIteratorSourceReader<>( + readerContext, new SourceRateLimiter(numbersPerSecond)); + } + } + + private static class ThrottlingIteratorSourceReader< + E, IterT extends Iterator, SplitT extends IteratorSourceSplit> + extends IteratorSourceReader { + private final SourceRateLimiter rateLimiter; + + public ThrottlingIteratorSourceReader( + SourceReaderContext context, SourceRateLimiter rateLimiter) { + super(context); + this.rateLimiter = rateLimiter; + } + + @Override + public InputStatus pollNext(ReaderOutput output) { + if (rateLimiter.request()) { + return super.pollNext(output); + } else { + return InputStatus.NOTHING_AVAILABLE; + } + } + } + + private static final class SourceRateLimiter { + private final AtomicBoolean newTokensAdded = new AtomicBoolean(false); + private final int tokensToAdd; + private int tokensAvailable; + + public SourceRateLimiter(int tokensPerSecond) { + this( + tokensPerSecond < 10 ? 1000 : 100, + tokensPerSecond < 10 ? tokensPerSecond : tokensPerSecond / 10); + } + + public SourceRateLimiter(int intervalMs, int tokensToAdd) { + checkArgument(intervalMs > 0); + checkArgument(tokensToAdd > 0); + this.tokensToAdd = tokensToAdd; + this.tokensAvailable = tokensToAdd; + new Timer("source-limiter", true) + .scheduleAtFixedRate( + new TimerTask() { + @Override + public void run() { + newTokensAdded.set(true); // "catch up" is ok + } + }, + intervalMs, + intervalMs); + } + + public boolean request() { + if (tokensAvailable == 0 && newTokensAdded.compareAndSet(true, false)) { + tokensAvailable = tokensToAdd; + } + if (tokensAvailable > 0) { + tokensAvailable--; + return true; + } else { + return false; + } + } + } + + private String checkpointAndCancel(JobID jobID) throws Exception { + waitForCheckpoint(jobID, cluster.getMiniCluster(), 1); + cluster.getClusterClient().cancel(jobID).get(); + checkStatus(jobID); + return CommonTestUtils.getLatestCompletedCheckpointPath(jobID, cluster.getMiniCluster()) + .orElseThrow( + () -> { + throw new NoSuchElementException("No checkpoint was created yet"); + }); + } + + private void checkStatus(JobID jobID) throws InterruptedException, ExecutionException { + if (cluster.getClusterClient().getJobStatus(jobID).get().isGloballyTerminalState()) { + cluster.getClusterClient() + .requestJobResult(jobID) + .get() + .getSerializedThrowable() + .ifPresent( + serializedThrowable -> { + throw new RuntimeException(serializedThrowable); + }); + } + } +} diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java b/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java index 66ce4a4bd3ca7..af58c57aad0cc 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java +++ b/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java @@ -19,6 +19,7 @@ package org.apache.flink.test.util; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.core.execution.JobClient; @@ -26,6 +27,7 @@ import org.apache.flink.runtime.checkpoint.metadata.CheckpointMetadata; import org.apache.flink.runtime.client.JobInitializationException; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation; import org.apache.flink.runtime.state.filesystem.AbstractFsCheckpointStorageAccess; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -35,11 +37,14 @@ import java.io.DataInputStream; import java.io.File; import java.io.IOException; +import java.io.UncheckedIOException; import java.nio.file.Files; +import java.nio.file.NoSuchFileException; import java.nio.file.Path; import java.nio.file.attribute.BasicFileAttributes; import java.util.Comparator; import java.util.Optional; +import java.util.concurrent.ExecutionException; import static org.apache.flink.runtime.state.filesystem.AbstractFsCheckpointStorageAccess.CHECKPOINT_DIR_PREFIX; import static org.apache.flink.runtime.state.filesystem.AbstractFsCheckpointStorageAccess.METADATA_FILE_NAME; @@ -111,11 +116,23 @@ public static CheckpointMetadata loadCheckpointMetadata(String savepointPath) } } + /** + * @deprecated please use {@link + * org.apache.flink.runtime.testutils.CommonTestUtils#getLatestCompletedCheckpointPath(JobID, + * MiniCluster)} which is less prone to {@link NoSuchFileException} and IO-intensive. + */ + @Deprecated public static File getMostRecentCompletedCheckpoint(File checkpointDir) throws IOException { return getMostRecentCompletedCheckpointMaybe(checkpointDir) .orElseThrow(() -> new IllegalStateException("Cannot generate checkpoint")); } + /** + * @deprecated please use {@link + * org.apache.flink.runtime.testutils.CommonTestUtils#getLatestCompletedCheckpointPath(JobID, + * MiniCluster)} which is less prone to {@link NoSuchFileException} and IO-intensive. + */ + @Deprecated public static Optional getMostRecentCompletedCheckpointMaybe(File checkpointDir) throws IOException { return Files.find(checkpointDir.toPath(), 2, TestUtils::isCompletedCheckpoint) @@ -138,9 +155,23 @@ private static boolean hasMetadata(Path file) { path.getFileName().toString().equals(METADATA_FILE_NAME)) .findAny() .isPresent(); - } catch (IOException e) { - ExceptionUtils.rethrow(e); + } catch (UncheckedIOException uncheckedIOException) { + // return false when the metadata file is in progress due to subsumed checkpoint + if (ExceptionUtils.findThrowable(uncheckedIOException, NoSuchFileException.class) + .isPresent()) { + return false; + } + throw uncheckedIOException; + } catch (IOException ioException) { + ExceptionUtils.rethrow(ioException); return false; // should never happen } } + + public static void waitUntilJobCanceled(JobID jobId, ClusterClient client) + throws ExecutionException, InterruptedException { + while (client.getJobStatus(jobId).get() != JobStatus.CANCELED) { + Thread.sleep(50); + } + } } diff --git a/flink-walkthroughs/flink-walkthrough-common/pom.xml b/flink-walkthroughs/flink-walkthrough-common/pom.xml index 15ed33e98eeb1..d2b7e68599aa3 100644 --- a/flink-walkthroughs/flink-walkthrough-common/pom.xml +++ b/flink-walkthroughs/flink-walkthrough-common/pom.xml @@ -27,7 +27,7 @@ under the License. org.apache.flink flink-walkthroughs - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-walkthroughs/flink-walkthrough-datastream-java/pom.xml b/flink-walkthroughs/flink-walkthrough-datastream-java/pom.xml index d5a29a2251485..36e58ed4a9b22 100644 --- a/flink-walkthroughs/flink-walkthrough-datastream-java/pom.xml +++ b/flink-walkthroughs/flink-walkthrough-datastream-java/pom.xml @@ -27,7 +27,7 @@ under the License. org.apache.flink flink-walkthroughs - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-walkthroughs/flink-walkthrough-datastream-scala/pom.xml b/flink-walkthroughs/flink-walkthrough-datastream-scala/pom.xml index fcd6c9460ac79..72b80c73465b8 100644 --- a/flink-walkthroughs/flink-walkthrough-datastream-scala/pom.xml +++ b/flink-walkthroughs/flink-walkthrough-datastream-scala/pom.xml @@ -27,7 +27,7 @@ under the License. org.apache.flink flink-walkthroughs - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-walkthroughs/pom.xml b/flink-walkthroughs/pom.xml index bc88b1de2bdd8..9ca2c705be64b 100644 --- a/flink-walkthroughs/pom.xml +++ b/flink-walkthroughs/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-yarn-tests/pom.xml b/flink-yarn-tests/pom.xml index 0bb1ac219b6a5..6dd8be9b5ef0c 100644 --- a/flink-yarn-tests/pom.xml +++ b/flink-yarn-tests/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java index 00882d079ed3a..308c51c80b7a9 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.client.deployment.ClusterDeploymentException; import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; @@ -76,7 +75,6 @@ import java.io.File; import java.io.IOException; import java.nio.charset.StandardCharsets; -import java.time.Duration; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -107,7 +105,6 @@ public class YARNHighAvailabilityITCase extends YarnTestBase { @ClassRule public static final TemporaryFolder FOLDER = new TemporaryFolder(); private static final String LOG_DIR = "flink-yarn-tests-ha"; - private static final Duration TIMEOUT = Duration.ofSeconds(200L); private static TestingServer zkServer; private static String storageDir; @@ -262,8 +259,7 @@ private void waitForApplicationAttempt(final ApplicationId applicationId, final yarnClient.getApplicationReport(applicationId); return applicationReport.getCurrentApplicationAttemptId().getAttemptId() >= attemptId; - }, - Deadline.fromNow(TIMEOUT)); + }); log.info("Attempt {} id detected.", attemptId); } @@ -319,8 +315,7 @@ private void killApplicationAndWait(final ApplicationId id) throws Exception { EnumSet.of( YarnApplicationState.KILLED, YarnApplicationState.FINISHED)) - .isEmpty(), - Deadline.fromNow(TIMEOUT)); + .isEmpty()); } private void waitForJobTermination( @@ -329,7 +324,7 @@ private void waitForJobTermination( log.info("Sending stop job signal"); stopJobSignal.signal(); final CompletableFuture jobResult = restClusterClient.requestJobResult(jobId); - jobResult.get(TIMEOUT.toMillis(), TimeUnit.MILLISECONDS); + jobResult.get(200, TimeUnit.SECONDS); } @Nonnull @@ -389,8 +384,7 @@ private void killApplicationMaster(final String processName) throws Exception { () -> { final Set curPids = getApplicationMasterPids(processName); return origPids.stream().noneMatch(curPids::contains); - }, - Deadline.fromNow(TIMEOUT)); + }); } private Set getApplicationMasterPids(final String processName) @@ -417,8 +411,7 @@ private static void waitUntilJobIsRunning( && jobDetails.getJobVertexInfos().stream() .map(toExecutionState()) .allMatch(isRunning()); - }, - Deadline.fromNow(TIMEOUT)); + }); } private static Function @@ -436,8 +429,7 @@ private static void waitUntilJobIsRestarted( final int expectedFullRestarts) throws Exception { CommonTestUtils.waitUntilCondition( - () -> getJobFullRestarts(restClusterClient, jobId) >= expectedFullRestarts, - Deadline.fromNow(TIMEOUT)); + () -> getJobFullRestarts(restClusterClient, jobId) >= expectedFullRestarts); } private static int getJobFullRestarts( diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java index ce71d3b8ae5c1..2089b5cbaa41d 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java @@ -18,7 +18,6 @@ package org.apache.flink.yarn; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.time.Time; import org.apache.flink.client.cli.CliFrontend; import org.apache.flink.configuration.Configuration; @@ -66,7 +65,6 @@ import java.io.IOException; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; -import java.time.Duration; import java.util.Arrays; import java.util.Collections; import java.util.Comparator; @@ -333,7 +331,7 @@ public void perJobYarnClusterOffHeap() throws Exception { // // Assert the number of TaskManager slots are set // - waitForTaskManagerRegistration(host, port, Duration.ofMillis(30_000)); + waitForTaskManagerRegistration(host, port); assertNumberOfSlotsPerTask(host, port, 3); final Map flinkConfig = getFlinkConfig(host, port); @@ -388,18 +386,16 @@ private void submitJob(final String jobFileName) throws IOException, Interrupted jobRunner.join(); } - private static void waitForTaskManagerRegistration( - final String host, final int port, final Duration waitDuration) throws Exception { - CommonTestUtils.waitUntilCondition( - () -> getNumberOfTaskManagers(host, port) > 0, Deadline.fromNow(waitDuration)); + private static void waitForTaskManagerRegistration(final String host, final int port) + throws Exception { + CommonTestUtils.waitUntilCondition(() -> getNumberOfTaskManagers(host, port) > 0); } private static void assertNumberOfSlotsPerTask( final String host, final int port, final int slotsNumber) throws Exception { try { CommonTestUtils.waitUntilCondition( - () -> getNumberOfSlotsPerTaskManager(host, port) == slotsNumber, - Deadline.fromNow(Duration.ofSeconds(30))); + () -> getNumberOfSlotsPerTaskManager(host, port) == slotsNumber); } catch (final TimeoutException e) { final int currentNumberOfSlots = getNumberOfSlotsPerTaskManager(host, port); fail( diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java index e080ae79a96b5..8356576b4bc42 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java @@ -18,7 +18,6 @@ package org.apache.flink.yarn; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.runtime.testutils.CommonTestUtils; @@ -45,7 +44,6 @@ import org.slf4j.event.Level; import java.io.File; -import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.EnumSet; @@ -150,19 +148,13 @@ ApplicationId runDetachedModeTest(Map securityProperties) throws jobRunner.join(); - final Duration timeout = Duration.ofMinutes(1); final long testConditionIntervalInMillis = 500; // in "new" mode we can only wait after the job is submitted, because TMs // are spun up lazily // wait until two containers are running LOG.info("Waiting until two containers are running"); CommonTestUtils.waitUntilCondition( - () -> getRunningContainers() >= 2, - Deadline.fromNow(timeout), - testConditionIntervalInMillis, - "We didn't reach the state of two containers running (instead: " - + getRunningContainers() - + ")"); + () -> getRunningContainers() >= 2, testConditionIntervalInMillis); LOG.info("Waiting until the job reaches FINISHED state"); final ApplicationId applicationId = getOnlyApplicationReport().getApplicationId(); @@ -172,11 +164,7 @@ ApplicationId runDetachedModeTest(Map securityProperties) throws new String[] {"switched from state RUNNING to FINISHED"}, applicationId, "jobmanager.log"), - Deadline.fromNow(timeout), - testConditionIntervalInMillis, - "The deployed job didn't finish on time reaching the timeout of " - + timeout - + " seconds. The application will be cancelled forcefully."); + testConditionIntervalInMillis); // kill application "externally". try { @@ -202,7 +190,6 @@ ApplicationId runDetachedModeTest(Map securityProperties) throws YarnApplicationState.KILLED, YarnApplicationState.FINISHED)) .isEmpty(), - Deadline.fromNow(timeout), testConditionIntervalInMillis); } catch (Throwable t) { LOG.warn("Killing failed", t); diff --git a/flink-yarn/pom.xml b/flink-yarn/pom.xml index 55ee45245ebda..20ab003d1590d 100644 --- a/flink-yarn/pom.xml +++ b/flink-yarn/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java index c99fd9c704fa4..92e94a12826a2 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java @@ -27,6 +27,7 @@ import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.deployment.application.ApplicationConfiguration; import org.apache.flink.client.program.ClusterClientProvider; +import org.apache.flink.client.program.PackagedProgramUtils; import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.ConfigOption; @@ -41,6 +42,7 @@ import org.apache.flink.configuration.ResourceManagerOptions; import org.apache.flink.configuration.RestOptions; import org.apache.flink.configuration.SecurityOptions; +import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.core.plugin.PluginConfig; import org.apache.flink.core.plugin.PluginUtils; import org.apache.flink.runtime.clusterframework.BootstrapTools; @@ -118,8 +120,10 @@ import java.util.Set; import java.util.stream.Collectors; +import static org.apache.flink.client.deployment.application.ApplicationConfiguration.APPLICATION_MAIN_CLASS; import static org.apache.flink.configuration.ConfigConstants.DEFAULT_FLINK_USR_LIB_DIR; import static org.apache.flink.configuration.ConfigConstants.ENV_FLINK_LIB_DIR; +import static org.apache.flink.configuration.ConfigConstants.ENV_FLINK_OPT_DIR; import static org.apache.flink.runtime.entrypoint.component.FileJobGraphRetriever.JOB_GRAPH_FILE_PATH; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -447,11 +451,15 @@ public ClusterClientProvider deployApplicationCluster( applicationConfiguration.applyToConfiguration(flinkConfiguration); - final List pipelineJars = - flinkConfiguration - .getOptional(PipelineOptions.JARS) - .orElse(Collections.emptyList()); - Preconditions.checkArgument(pipelineJars.size() == 1, "Should only have one jar"); + // No need to do pipelineJars validation if it is a PyFlink job. + if (!(PackagedProgramUtils.isPython(applicationConfiguration.getApplicationClassName()) + || PackagedProgramUtils.isPython(applicationConfiguration.getProgramArguments()))) { + final List pipelineJars = + flinkConfiguration + .getOptional(PipelineOptions.JARS) + .orElse(Collections.emptyList()); + Preconditions.checkArgument(pipelineJars.size() == 1, "Should only have one jar"); + } try { return deployInternal( @@ -910,6 +918,17 @@ private ApplicationReport startAppMaster( LocalResourceType.ARCHIVE); } + // only for application mode + // Python jar file only needs to be shipped and should not be added to classpath. + if (YarnApplicationClusterEntryPoint.class.getName().equals(yarnClusterEntrypoint) + && PackagedProgramUtils.isPython(configuration.get(APPLICATION_MAIN_CLASS))) { + fileUploader.registerMultipleLocalResources( + Collections.singletonList( + new Path(PackagedProgramUtils.getPythonJar().toURI())), + ConfigConstants.DEFAULT_FLINK_OPT_DIR, + LocalResourceType.FILE); + } + // Upload and register user jars final List userClassPaths = fileUploader.registerMultipleLocalResources( @@ -996,6 +1015,13 @@ private ApplicationReport startAppMaster( File tmpConfigurationFile = null; try { tmpConfigurationFile = File.createTempFile(appId + "-flink-conf.yaml", null); + + // remove localhost bind hosts as they render production clusters unusable + removeLocalhostBindHostSetting(configuration, JobManagerOptions.BIND_HOST); + removeLocalhostBindHostSetting(configuration, TaskManagerOptions.BIND_HOST); + // this setting is unconditionally overridden anyway, so we remove it for clarity + configuration.removeConfig(TaskManagerOptions.HOST); + BootstrapTools.writeConfiguration(configuration, tmpConfigurationFile); String flinkConfigKey = "flink-conf.yaml"; @@ -1134,6 +1160,10 @@ private ApplicationReport startAppMaster( // set Flink app class path appMasterEnv.put(YarnConfigKeys.ENV_FLINK_CLASSPATH, classPathBuilder.toString()); + // Set FLINK_OPT_DIR to `opt` folder under working dir in container + appMasterEnv.put( + ENV_FLINK_OPT_DIR, Path.CUR_DIR + "/" + ConfigConstants.DEFAULT_FLINK_OPT_DIR); + // set Flink on YARN internal configuration values appMasterEnv.put(YarnConfigKeys.FLINK_DIST_JAR, localResourceDescFlinkJar.toString()); appMasterEnv.put(YarnConfigKeys.ENV_APP_ID, appId.toString()); @@ -1260,6 +1290,20 @@ private ApplicationReport startAppMaster( return report; } + private void removeLocalhostBindHostSetting( + Configuration configuration, ConfigOption option) { + configuration + .getOptional(option) + .filter(bindHost -> bindHost.equals("localhost")) + .ifPresent( + bindHost -> { + LOG.info( + "Removing 'localhost' {} setting from effective configuration; using '0.0.0.0' instead.", + option); + configuration.removeConfig(option); + }); + } + /** * Returns the configured remote target home directory if set, otherwise returns the default * home directory. diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnApplicationClusterEntryPoint.java b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnApplicationClusterEntryPoint.java index d38ebaeb666f5..d619d370f5cfb 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnApplicationClusterEntryPoint.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnApplicationClusterEntryPoint.java @@ -24,6 +24,7 @@ import org.apache.flink.client.program.DefaultPackagedProgramRetriever; import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.client.program.PackagedProgramRetriever; +import org.apache.flink.client.program.PackagedProgramUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.PipelineOptions; import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; @@ -132,9 +133,17 @@ private static PackagedProgramRetriever getPackagedProgramRetriever( throws FlinkException { final File userLibDir = YarnEntrypointUtils.getUsrLibDir(configuration).orElse(null); - final File userApplicationJar = getUserApplicationJar(userLibDir, configuration); + + // No need to do pipelineJars validation if it is a PyFlink job. + if (!(PackagedProgramUtils.isPython(jobClassName) + || PackagedProgramUtils.isPython(programArguments))) { + final File userApplicationJar = getUserApplicationJar(userLibDir, configuration); + return DefaultPackagedProgramRetriever.create( + userLibDir, userApplicationJar, jobClassName, programArguments, configuration); + } + return DefaultPackagedProgramRetriever.create( - userLibDir, userApplicationJar, jobClassName, programArguments, configuration); + userLibDir, jobClassName, programArguments, configuration); } private static File getUserApplicationJar( diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java index 986fd3785814c..2321355a390bc 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java @@ -25,7 +25,6 @@ import org.apache.flink.configuration.ResourceManagerOptions; import org.apache.flink.configuration.RestOptions; import org.apache.flink.configuration.SecurityOptions; -import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.configuration.WebOptions; import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.util.Preconditions; @@ -65,9 +64,6 @@ public static Configuration loadConfiguration( ApplicationConstants.Environment.NM_HOST.key()); configuration.setString(JobManagerOptions.ADDRESS, hostname); - configuration.removeConfig(JobManagerOptions.BIND_HOST); - configuration.removeConfig(TaskManagerOptions.BIND_HOST); - configuration.removeConfig(TaskManagerOptions.HOST); configuration.setString(RestOptions.ADDRESS, hostname); configuration.setString(RestOptions.BIND_ADDRESS, hostname); diff --git a/pom.xml b/pom.xml index 5e4498125db7c..1c79a73b08ccd 100644 --- a/pom.xml +++ b/pom.xml @@ -28,7 +28,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 Flink : pom @@ -544,7 +544,7 @@ under the License. jackson-bom pom import - 2.13.0 + 2.13.2.20220328 diff --git a/scripts/dependencies.json b/scripts/dependencies.json new file mode 100644 index 0000000000000..e03c3f1e4f077 --- /dev/null +++ b/scripts/dependencies.json @@ -0,0 +1,12 @@ +[ + { + "name": "flink", + "dependencies": [ + { + "repo_owner": "streamnative", + "repo_name": "streamnative-sql-gateway", + "tag": "0.6.1-SNAPSHOT" + } + ] + } +] diff --git a/tools/azure-pipelines/e2e-template.yml b/tools/azure-pipelines/e2e-template.yml index f590c15758c1c..637ea476d30fd 100644 --- a/tools/azure-pipelines/e2e-template.yml +++ b/tools/azure-pipelines/e2e-template.yml @@ -102,8 +102,8 @@ jobs: echo "Installing required software" sudo apt-get install -y bc libapr1 # install libssl1.0.0 for netty tcnative - wget http://security.ubuntu.com/ubuntu/pool/main/o/openssl1.0/libssl1.0.0_1.0.2n-1ubuntu5.7_amd64.deb - sudo apt install ./libssl1.0.0_1.0.2n-1ubuntu5.7_amd64.deb + wget http://security.ubuntu.com/ubuntu/pool/main/o/openssl1.0/libssl1.0.0_1.0.2n-1ubuntu5.8_amd64.deb + sudo apt install ./libssl1.0.0_1.0.2n-1ubuntu5.8_amd64.deb displayName: Prepare E2E run condition: not(eq(variables['SKIP'], '1')) - script: ${{parameters.environment}} PROFILE="$PROFILE -Dfast -Pskip-webui-build" ./tools/ci/compile.sh @@ -128,4 +128,4 @@ jobs: - script: ./tools/azure-pipelines/cache_docker_images.sh save displayName: Save docker images condition: and(not(eq(variables['SKIP'], '1')), ne(variables.DOCKER_IMAGES_CACHE_HIT, 'true')) - continueOnError: true \ No newline at end of file + continueOnError: true diff --git a/tools/ci/compile.sh b/tools/ci/compile.sh index c98fbae3f4af6..e6013f9650f3e 100755 --- a/tools/ci/compile.sh +++ b/tools/ci/compile.sh @@ -108,10 +108,6 @@ check_shaded_artifacts_connector_elasticsearch 6 EXIT_CODE=$(($EXIT_CODE+$?)) check_shaded_artifacts_connector_elasticsearch 7 EXIT_CODE=$(($EXIT_CODE+$?)) -check_one_per_package_file_connector_base -EXIT_CODE=$(($EXIT_CODE+$?)) -check_relocated_file_connector_base -EXIT_CODE=$(($EXIT_CODE+$?)) echo "============ Run license check ============" diff --git a/tools/ci/java-ci-tools/pom.xml b/tools/ci/java-ci-tools/pom.xml index a749bb48ede33..a82f032113106 100644 --- a/tools/ci/java-ci-tools/pom.xml +++ b/tools/ci/java-ci-tools/pom.xml @@ -25,12 +25,12 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 ../../.. java-ci-tools - 1.15-SNAPSHOT + 1.15.0 Flink : Tools : CI : Java diff --git a/tools/ci/log4j.properties b/tools/ci/log4j.properties index bf831743e6f26..80a611405d7e4 100644 --- a/tools/ci/log4j.properties +++ b/tools/ci/log4j.properties @@ -61,6 +61,11 @@ logger.zookeeper.name = org.apache.zookeeper logger.zookeeper.level = INFO logger.zookeeper.additivity = false logger.zookeeper.appenderRef.zk.ref = ZooKeeperServerAppender +# FinalRequestProcessor in DEBUG mode enables us to see requests being processed by the ZK server +logger.zookeeper_FinalRequestProcessor.name = org.apache.zookeeper.server.FinalRequestProcessor +logger.zookeeper_FinalRequestProcessor.level = DEBUG +logger.zookeeper_FinalRequestProcessor.additivity = false +logger.zookeeper_FinalRequestProcessor.appenderRef.zk.ref = ZooKeeperServerAppender logger.shaded_zookeeper.name = org.apache.flink.shaded.zookeeper3 logger.shaded_zookeeper.level = INFO logger.shaded_zookeeper.additivity = false diff --git a/tools/ci/shade.sh b/tools/ci/shade.sh index 5931faffea719..922bba42f04f5 100755 --- a/tools/ci/shade.sh +++ b/tools/ci/shade.sh @@ -153,7 +153,7 @@ check_shaded_artifacts_connector_elasticsearch() { VARIANT=$1 find flink-connectors/flink-connector-elasticsearch${VARIANT}/target/flink-connector-elasticsearch${VARIANT}*.jar ! -name "*-tests.jar" -exec jar tf {} \; > allClasses - UNSHADED_CLASSES=`cat allClasses | grep -v -e '^META-INF' -e '^assets' -e "^org/apache/flink/connector/elasticsearch/" -e "^org/apache/flink/streaming/connectors/elasticsearch/" -e "^org/apache/flink/streaming/connectors/elasticsearch${VARIANT}/" -e "^org/apache/flink/connector/elasticsearch${VARIANT}/shaded/" -e "^org/apache/flink/table/descriptors/" -e "^org/elasticsearch/" | grep '\.class$'` + UNSHADED_CLASSES=`cat allClasses | grep -v -e '^META-INF' -e '^assets' -e "^org/apache/flink/connector/elasticsearch/" -e "^org/apache/flink/streaming/connectors/elasticsearch/" -e "^org/apache/flink/streaming/connectors/elasticsearch${VARIANT}/" -e "^org/apache/flink/table/descriptors/" -e "^org/elasticsearch/" | grep '\.class$'` if [ "$?" = "0" ]; then echo "==============================================================================" echo "Detected unshaded dependencies in flink-connector-elasticsearch${VARIANT}'s fat jar:" @@ -173,62 +173,3 @@ check_shaded_artifacts_connector_elasticsearch() { return 0 } - -check_one_per_package() { - read foo - if [ $foo -gt 1 ] - then - echo "ERROR - CHECK FAILED: $1 is shaded multiple times!" - exit 1 - else - echo "OK" - fi -} - -check_relocated() { - read foo - if [ $foo -ne 0 ] - then - echo "ERROR - CHECK FAILED: found $1 classes that where not relocated!" - exit 1 - else - echo "OK" - fi -} - -check_one_per_package_file_connector_base() { - echo "Checking that flink-connector-base is included only once:" - echo "__________________________________________________________________________" - - CONNECTOR_JARS=$(find flink-connectors -type f -name '*.jar' | grep -vE "original|connector-hive" | grep -v '\-test'); - EXIT_CODE=0 - - for i in $CONNECTOR_JARS; - do - echo -n "- $i: "; - jar tf $i | grep 'org/apache/flink/connector/base/source/reader/RecordEmitter' | wc -l | check_one_per_package "flink-connector-base"; - EXIT_CODE=$((EXIT_CODE+$?)) - done; - return $EXIT_CODE; -} - -check_relocated_file_connector_base() { - echo -e "\n\n" - echo "Checking that flink-connector-base is relocated:" - echo "__________________________________________________________________________" - - CONNECTOR_JARS=$(find flink-connectors -type f -name '*.jar' | \ - grep -v original | grep -v '\-test' | grep -v 'flink-connectors/flink-connector-base'); - - EXIT_CODE=0 - for i in $CONNECTOR_JARS; - do - echo -n "- $i: "; - jar tf $i | grep '^org/apache/flink/connector/base/source/reader/RecordEmitter' | wc -l | check_relocated "flink-connector-base"; - EXIT_CODE=$((EXIT_CODE+$?)) - done; - return $EXIT_CODE; -} - - - diff --git a/tools/releasing/create_snapshot_branch.sh b/tools/releasing/create_snapshot_branch.sh index c3d86747299a0..a010001bcc560 100755 --- a/tools/releasing/create_snapshot_branch.sh +++ b/tools/releasing/create_snapshot_branch.sh @@ -55,7 +55,7 @@ perl -pi -e "s#^baseURL = .*#baseURL = \'${url_base}${SHORT_RELEASE_VERSION}\'#" perl -pi -e "s#^ JavaDocs = .*# JavaDocs = \"${url_base}${SHORT_RELEASE_VERSION}/api/java/\"#" ${config_file} perl -pi -e "s#^ \[\"JavaDocs\", .*# \[\"JavaDocs\", \"${url_base}${SHORT_RELEASE_VERSION}/api/java/\"\],#" ${config_file} perl -pi -e "s#^ ScalaDocs = .*# ScalaDocs = \"${url_base}${SHORT_RELEASE_VERSION}/api/scala/index.html\#org.apache.flink.api.scala.package\"#" ${config_file} -perl -pi -e "s#^ \[\"ScalaDocs\", .*# \[\"ScalaDocs\", \"${url_base}${SHORT_RELEASE_VERSION}/api/scala/index.html\#org.apache.flink.api.scala.package/\"\]#" ${config_file} +perl -pi -e "s#^ \[\"ScalaDocs\", .*# \[\"ScalaDocs\", \"${url_base}${SHORT_RELEASE_VERSION}/api/scala/index.html\#org.apache.flink.api.scala.package/\"\],#" ${config_file} perl -pi -e "s#^ PyDocs = .*# PyDocs = \"${url_base}${SHORT_RELEASE_VERSION}/api/python/\"#" ${config_file} perl -pi -e "s#^ \[\"PyDocs\", .*# \[\"PyDocs\", \"${url_base}${SHORT_RELEASE_VERSION}/api/python/\"\]#" ${config_file}