(flink) branch master updated: [FLINK-35735][sql-gateway] Don't list all catalogs when closing session (#25010)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new af3e39fa4ce [FLINK-35735][sql-gateway] Don't list all catalogs when closing session (#25010) af3e39fa4ce is described below commit af3e39fa4ce70187166e2f1dfef1ebfeee6d40cb Author: Xuyang AuthorDate: Wed Jul 3 10:13:02 2024 +0800 [FLINK-35735][sql-gateway] Don't list all catalogs when closing session (#25010) --- .../table/gateway/service/context/SessionContext.java| 16 +++- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/context/SessionContext.java b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/context/SessionContext.java index cf1597ecea9..12bfc2d8e5a 100644 --- a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/context/SessionContext.java +++ b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/context/SessionContext.java @@ -255,15 +255,13 @@ public class SessionContext { /** Close resources, e.g. catalogs. */ public void close() { operationManager.close(); -for (String name : sessionState.catalogManager.listCatalogs()) { -try { - sessionState.catalogManager.getCatalog(name).ifPresent(Catalog::close); -} catch (Throwable t) { -LOG.error( -String.format( -"Failed to close catalog %s for the session %s.", name, sessionId), -t); -} + +try { +sessionState.catalogManager.close(); +} catch (Throwable t) { +LOG.error( +String.format("Failed to close catalog manager for the session %s.", sessionId), +t); } try { userClassloader.close();
(flink) branch master updated (922cc2ad522 -> 1070c6e9e0f)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 922cc2ad522 [FLINK-25537] [JUnit5 Migration] Module: flink-core package api-common (#23960) add 1070c6e9e0f [FLINK-34312][table] Improve the handling of default node types for named parameters (#24235) No new revisions were added by this update. Summary of changes: .../calcite/sql/validate/ProcedureNamespace.java | 5 +- .../calcite/sql/validate/SqlValidatorImpl.java | 20 ++- .../apache/calcite/sql2rel/SqlToRelConverter.java | 37 ++--- .../planner/calcite/FlinkConvertletTable.java | 39 - .../planner/calcite/FlinkOperatorBinding.java | 181 - .../table/planner/calcite/FlinkSqlCallBinding.java | 117 + .../inference/CallBindingCallContext.java | 16 +- .../inference/OperatorBindingCallContext.java | 9 +- .../functions/sql/FlinkSqlOperatorTable.java | 6 - .../planner/functions/sql/SqlDefaultOperator.java | 58 +++ .../converters/SqlProcedureCallConverter.java | 37 ++--- .../planner/codegen/calls/StringCallGen.scala | 12 +- 12 files changed, 240 insertions(+), 297 deletions(-) delete mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/FlinkOperatorBinding.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/FlinkSqlCallBinding.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/SqlDefaultOperator.java
(flink) branch release-1.19 updated: [FLINK-34265][doc] Add the doc of named parameters (#24377)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch release-1.19 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.19 by this push: new 0af2540dc30 [FLINK-34265][doc] Add the doc of named parameters (#24377) 0af2540dc30 is described below commit 0af2540dc30340742506b3c61850f1e2d25f4d72 Author: Feng Jin AuthorDate: Mon Feb 26 19:25:43 2024 +0800 [FLINK-34265][doc] Add the doc of named parameters (#24377) --- docs/content.zh/docs/dev/table/functions/udfs.md | 139 + docs/content.zh/docs/dev/table/procedures.md | 146 +++ docs/content/docs/dev/table/functions/udfs.md| 137 + docs/content/docs/dev/table/procedures.md| 144 ++ 4 files changed, 566 insertions(+) diff --git a/docs/content.zh/docs/dev/table/functions/udfs.md b/docs/content.zh/docs/dev/table/functions/udfs.md index 5527cfcb978..f137b2ba446 100644 --- a/docs/content.zh/docs/dev/table/functions/udfs.md +++ b/docs/content.zh/docs/dev/table/functions/udfs.md @@ -77,6 +77,9 @@ env.from("MyTable").select(call("SubstringFunction", $("myField"), 5, 12)); // 在 SQL 里调用注册好的函数 env.sqlQuery("SELECT SubstringFunction(myField, 5, 12) FROM MyTable"); +// 在 SQL 里使用命名参数调用注册好的函数 +env.sqlQuery("SELECT SubstringFunction(param1 => myField, param2 => 5, param3 => 12) FROM MyTable"); + ``` {{< /tab >}} {{< tab "Scala" >}} @@ -599,6 +602,142 @@ public static class LiteralFunction extends ScalarFunction { For more examples of custom type inference, see also the `flink-examples-table` module with {{< gh_link file="flink-examples/flink-examples-table/src/main/java/org/apache/flink/table/examples/java/functions/AdvancedFunctionsExample.java" name="advanced function implementation" >}}. +### 命名参数 + +在调用函数时,可以使用参数名称来指定参数值。命名参数允许同时传递参数名和值给函数,避免了因为错误的参数顺序而导致混淆,并提高了代码的可读性和可维护性。 此外,命名参数还可以省略非必需的参数,默认情况下会使用 `null` 进行填充。 +我们可以通过 `@ArgumentHint` 注解来指定参数的名称,类型,是否是必需的参数等。 + +**`@ArgumentHint`** + +下面三个示例展示了如何在不同的范围内使用 `@ArgumentHint`。更多信息请参考注解类的文档。 + +1. 在 function 的 `eval` 方法的参数上使用 `@ArgumentHint` 注解。 + +{{< tabs "8064df87-eb42-4def-9bd2-0988fc246d37" >}} +{{< tab "Java" >}} + +```java +import com.sun.tracing.dtrace.ArgsAttributes; +import org.apache.flink.table.annotation.ArgumentHint; +import org.apache.flink.table.functions.ScalarFunction; + +public static class NamedParameterClass extends ScalarFunction { + +// 使用 @ArgumentHint 注解指定参数的名称,参数类型,以及是否是必需的参数 +public String eval(@ArgumentHint(name = "param1", isOptional = false, type = @DataTypeHint("STRING")) String s1, + @ArgumentHint(name = "param2", isOptional = true, type = @DataTypeHint("INT")) Integer s2) { +return s1 + ", " + s2; +} +} + +``` +{{< /tab >}} +{{< tab "Scala" >}} +```scala +import org.apache.flink.table.annotation.ArgumentHint; +import org.apache.flink.table.functions.ScalarFunction; + +class NamedParameterClass extends ScalarFunction { + + // 使用 @ArgumentHint 注解指定参数的名称,参数类型,以及是否是必需的参数 + def eval(@ArgumentHint(name = "param1", isOptional = false, `type` = @DataTypeHint("STRING")) s1: String, + @ArgumentHint(name = "param2", isOptional = true, `type` = @DataTypeHint("INTEGER")) s2: Integer) = { +s1 + ", " + s2 + } +} +``` +{{< /tab >}} +{{< /tabs >}} + +2. 在 function 的 `eval` 方法上使用 `@ArgumentHint` 注解。 + +{{< tabs "1356086c-189c-4932-a797-badf5b5e27ab" >}} +{{< tab "Java" >}} +```java +import org.apache.flink.table.annotation.ArgumentHint; +import org.apache.flink.table.functions.ScalarFunction; + +public static class NamedParameterClass extends ScalarFunction { + + // 使用 @ArgumentHint 注解指定参数的名称,参数类型,以及该参数是否是必需的参数 + @FunctionHint( + argument = {@ArgumentHint(name = "param1", isOptional = false, type = @DataTypeHint("STRING")), + @ArgumentHint(name = "param2", isOptional = true, type = @DataTypeHint("INTEGER"))} + ) + public String eval(String s1, Integer s2) { +return s1 + ", " + s2; + } +} +``` +{{< /tab >}} +{{< tab "Scala" >}} +```scala +import org.apache.flink.table.annotation.ArgumentHint; +import org.apache.flink.table.functions.ScalarFunction; + +class NamedParameterClass extends ScalarFunction { + + // 使用 @ArgumentHint 注解指定参数的名称,参数类型,以及是否是必需的参数 + @FunctionHint( +argument = Array(new ArgumentHint(name = "param1", isOptional = false, `type` = new DataTypeHint("STRING")), + new ArgumentHint(name = "param2", isOptional = true, `type` = new Da
(flink) branch master updated: [FLINK-34265][doc] Add the doc of named parameters (#24260)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new 26b1d1bbff5 [FLINK-34265][doc] Add the doc of named parameters (#24260) 26b1d1bbff5 is described below commit 26b1d1bbff590589c72af892fc22f80fa4ee1261 Author: Feng Jin AuthorDate: Mon Feb 26 13:21:01 2024 +0800 [FLINK-34265][doc] Add the doc of named parameters (#24260) --- docs/content.zh/docs/dev/table/functions/udfs.md | 139 + docs/content.zh/docs/dev/table/procedures.md | 146 +++ docs/content/docs/dev/table/functions/udfs.md| 137 + docs/content/docs/dev/table/procedures.md| 144 ++ 4 files changed, 566 insertions(+) diff --git a/docs/content.zh/docs/dev/table/functions/udfs.md b/docs/content.zh/docs/dev/table/functions/udfs.md index 5527cfcb978..f137b2ba446 100644 --- a/docs/content.zh/docs/dev/table/functions/udfs.md +++ b/docs/content.zh/docs/dev/table/functions/udfs.md @@ -77,6 +77,9 @@ env.from("MyTable").select(call("SubstringFunction", $("myField"), 5, 12)); // 在 SQL 里调用注册好的函数 env.sqlQuery("SELECT SubstringFunction(myField, 5, 12) FROM MyTable"); +// 在 SQL 里使用命名参数调用注册好的函数 +env.sqlQuery("SELECT SubstringFunction(param1 => myField, param2 => 5, param3 => 12) FROM MyTable"); + ``` {{< /tab >}} {{< tab "Scala" >}} @@ -599,6 +602,142 @@ public static class LiteralFunction extends ScalarFunction { For more examples of custom type inference, see also the `flink-examples-table` module with {{< gh_link file="flink-examples/flink-examples-table/src/main/java/org/apache/flink/table/examples/java/functions/AdvancedFunctionsExample.java" name="advanced function implementation" >}}. +### 命名参数 + +在调用函数时,可以使用参数名称来指定参数值。命名参数允许同时传递参数名和值给函数,避免了因为错误的参数顺序而导致混淆,并提高了代码的可读性和可维护性。 此外,命名参数还可以省略非必需的参数,默认情况下会使用 `null` 进行填充。 +我们可以通过 `@ArgumentHint` 注解来指定参数的名称,类型,是否是必需的参数等。 + +**`@ArgumentHint`** + +下面三个示例展示了如何在不同的范围内使用 `@ArgumentHint`。更多信息请参考注解类的文档。 + +1. 在 function 的 `eval` 方法的参数上使用 `@ArgumentHint` 注解。 + +{{< tabs "8064df87-eb42-4def-9bd2-0988fc246d37" >}} +{{< tab "Java" >}} + +```java +import com.sun.tracing.dtrace.ArgsAttributes; +import org.apache.flink.table.annotation.ArgumentHint; +import org.apache.flink.table.functions.ScalarFunction; + +public static class NamedParameterClass extends ScalarFunction { + +// 使用 @ArgumentHint 注解指定参数的名称,参数类型,以及是否是必需的参数 +public String eval(@ArgumentHint(name = "param1", isOptional = false, type = @DataTypeHint("STRING")) String s1, + @ArgumentHint(name = "param2", isOptional = true, type = @DataTypeHint("INT")) Integer s2) { +return s1 + ", " + s2; +} +} + +``` +{{< /tab >}} +{{< tab "Scala" >}} +```scala +import org.apache.flink.table.annotation.ArgumentHint; +import org.apache.flink.table.functions.ScalarFunction; + +class NamedParameterClass extends ScalarFunction { + + // 使用 @ArgumentHint 注解指定参数的名称,参数类型,以及是否是必需的参数 + def eval(@ArgumentHint(name = "param1", isOptional = false, `type` = @DataTypeHint("STRING")) s1: String, + @ArgumentHint(name = "param2", isOptional = true, `type` = @DataTypeHint("INTEGER")) s2: Integer) = { +s1 + ", " + s2 + } +} +``` +{{< /tab >}} +{{< /tabs >}} + +2. 在 function 的 `eval` 方法上使用 `@ArgumentHint` 注解。 + +{{< tabs "1356086c-189c-4932-a797-badf5b5e27ab" >}} +{{< tab "Java" >}} +```java +import org.apache.flink.table.annotation.ArgumentHint; +import org.apache.flink.table.functions.ScalarFunction; + +public static class NamedParameterClass extends ScalarFunction { + + // 使用 @ArgumentHint 注解指定参数的名称,参数类型,以及该参数是否是必需的参数 + @FunctionHint( + argument = {@ArgumentHint(name = "param1", isOptional = false, type = @DataTypeHint("STRING")), + @ArgumentHint(name = "param2", isOptional = true, type = @DataTypeHint("INTEGER"))} + ) + public String eval(String s1, Integer s2) { +return s1 + ", " + s2; + } +} +``` +{{< /tab >}} +{{< tab "Scala" >}} +```scala +import org.apache.flink.table.annotation.ArgumentHint; +import org.apache.flink.table.functions.ScalarFunction; + +class NamedParameterClass extends ScalarFunction { + + // 使用 @ArgumentHint 注解指定参数的名称,参数类型,以及是否是必需的参数 + @FunctionHint( +argument = Array(new ArgumentHint(name = "param1", isOptional = false, `type` = new DataTypeHint("STRING")), + new ArgumentHint(name = "param2", isOptional = true, `type` = new DataTypeHint(&
(flink) branch master updated: [FLINK-33358][sql-client] Fix Flink SQL Client fail to start in Flink on YARN (#23629)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new fa364c7c668 [FLINK-33358][sql-client] Fix Flink SQL Client fail to start in Flink on YARN (#23629) fa364c7c668 is described below commit fa364c7c668ec2a87bcbf18ce2b80f749cc16b2b Author: Prabhu Joseph AuthorDate: Thu Feb 1 09:21:40 2024 +0530 [FLINK-33358][sql-client] Fix Flink SQL Client fail to start in Flink on YARN (#23629) Co-authored-by: Prabhu Joseph --- flink-table/flink-sql-client/bin/sql-client.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-table/flink-sql-client/bin/sql-client.sh b/flink-table/flink-sql-client/bin/sql-client.sh index 3091bdf1b36..797dcc4ac05 100755 --- a/flink-table/flink-sql-client/bin/sql-client.sh +++ b/flink-table/flink-sql-client/bin/sql-client.sh @@ -87,7 +87,7 @@ if [[ "$CC_CLASSPATH" =~ .*flink-sql-client.*.jar ]]; then elif [ -n "$FLINK_SQL_CLIENT_JAR" ]; then # start client with jar -exec "$JAVA_RUN" $FLINK_ENV_JAVA_OPTS $JVM_ARGS "${log_setting[@]}" -classpath "`manglePathList "$CC_CLASSPATH:$INTERNAL_HADOOP_CLASSPATHS:$FLINK_SQL_CLIENT_JAR"`" org.apache.flink.table.client.SqlClient "$@" --jar "`manglePath $FLINK_SQL_CLIENT_JAR`" +exec "$JAVA_RUN" $FLINK_ENV_JAVA_OPTS $JVM_ARGS "${log_setting[@]}" -classpath "`manglePathList "$CC_CLASSPATH:$FLINK_SQL_CLIENT_JAR:$INTERNAL_HADOOP_CLASSPATHS"`" org.apache.flink.table.client.SqlClient "$@" --jar "`manglePath $FLINK_SQL_CLIENT_JAR`" # write error message to stderr else
(flink) branch master updated: [FLINK-34058][table] Support optional parameters for named arguments (#24183)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new 4cd43fc09bd [FLINK-34058][table] Support optional parameters for named arguments (#24183) 4cd43fc09bd is described below commit 4cd43fc09bd6c2e4806792fa2cce71f54ec1a9dd Author: Feng Jin AuthorDate: Mon Jan 29 23:40:07 2024 +0800 [FLINK-34058][table] Support optional parameters for named arguments (#24183) - Co-authored-by: xuyang Co-authored-by: Shengkai <1059623...@qq.com> --- .../types/extraction/BaseMappingExtractor.java | 51 ++- .../extraction/FunctionSignatureTemplate.java | 30 +- .../table/types/extraction/FunctionTemplate.java | 15 +- .../types/extraction/TypeInferenceExtractor.java | 22 +- .../flink/table/types/inference/TypeInference.java | 26 ++ .../extraction/TypeInferenceExtractorTest.java | 347 - .../apache/calcite/sql2rel/SqlToRelConverter.java | 21 +- .../planner/calcite/FlinkConvertletTable.java | 39 +++ .../planner/calcite/FlinkOperatorBinding.java | 181 +++ .../inference/CallBindingCallContext.java | 21 +- .../inference/OperatorBindingCallContext.java | 31 +- .../inference/TypeInferenceOperandChecker.java | 41 ++- .../functions/sql/FlinkSqlOperatorTable.java | 6 + .../operations/PlannerCallProcedureOperation.java | 4 +- .../converters/SqlProcedureCallConverter.java | 36 ++- .../planner/codegen/calls/StringCallGen.scala | 5 +- .../factories/TestProcedureCatalogFactory.java | 18 ++ .../planner/runtime/stream/sql/FunctionITCase.java | 140 - .../runtime/stream/sql/ProcedureITCase.java| 20 +- 19 files changed, 976 insertions(+), 78 deletions(-) diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/BaseMappingExtractor.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/BaseMappingExtractor.java index fdd12e1d795..6069a0f729c 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/BaseMappingExtractor.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/BaseMappingExtractor.java @@ -34,6 +34,7 @@ import java.lang.reflect.Method; import java.lang.reflect.Parameter; import java.lang.reflect.ParameterizedType; import java.lang.reflect.Type; +import java.util.Arrays; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -143,6 +144,9 @@ abstract class BaseMappingExtractor { // check if the method can be called verifyMappingForMethod(correctMethod, collectedMappingsPerMethod, verification); +// check if we declare optional on a primitive type parameter +verifyOptionalOnPrimitiveParameter(correctMethod, collectedMappingsPerMethod); + // check if method strategies conflict with function strategies collectedMappingsPerMethod.forEach( (signature, result) -> putMapping(collectedMappings, signature, result)); @@ -323,6 +327,40 @@ abstract class BaseMappingExtractor { verification.verify(method, signature.toClass(), result.toClass())); } +private void verifyOptionalOnPrimitiveParameter( +Method method, +Map collectedMappingsPerMethod) { +collectedMappingsPerMethod +.keySet() +.forEach( +signature -> { +Boolean[] argumentOptional = signature.argumentOptionals; +// Here we restrict that the argument must contain optional parameters +// in order to obtain the FunctionSignatureTemplate of the method for +// verification. Therefore, the extract method will only be called once. +// If no function hint is set, this verify will not be executed. +if (argumentOptional != null +&& Arrays.stream(argumentOptional) +.anyMatch(Boolean::booleanValue)) { +FunctionSignatureTemplate functionResultTemplate = +signatureExtraction.extract(this, method); +for (int i = 0; i < argumentOptional.length; i++) { +DataType dataType = + functionResultTemplate.argumentTemplates.get(i) +.dataType; +
(flink) branch master updated (de7322112cb -> eb848dc0676)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from de7322112cb [FLINK-34048][table] introduce unslicing window agg for session window tvf new e1ebcaff52f [FLINK-34055][table] Introduce a new annotation for named parameters new 9171194ef26 [FLINK-34057][table] Support named parameters for functions new eb848dc0676 [FLINK-34056][table] Support named parameters for procedures The 3 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference. Summary of changes: .../flink/table/annotation/ArgumentHint.java | 69 ++ .../flink/table/annotation/FunctionHint.java | 29 +++ .../flink/table/annotation/ProcedureHint.java | 29 +++ .../types/extraction/BaseMappingExtractor.java | 14 +- .../table/types/extraction/DataTypeExtractor.java | 9 +- .../table/types/extraction/DataTypeTemplate.java | 9 + .../table/types/extraction/ExtractionUtils.java| 13 +- .../extraction/FunctionSignatureTemplate.java | 6 + .../table/types/extraction/FunctionTemplate.java | 53 - .../types/extraction/TypeInferenceExtractor.java | 4 +- .../extraction/TypeInferenceExtractorTest.java | 233 - .../calcite/sql/validate/SqlValidatorImpl.java | 15 +- .../apache/calcite/sql2rel/SqlToRelConverter.java | 22 +- .../planner/calcite/RexSetSemanticsTableCall.java | 6 + .../inference/OperatorBindingCallContext.java | 22 +- .../inference/TypeInferenceOperandChecker.java | 44 +++- .../converters/SqlProcedureCallConverter.java | 13 +- .../logical/FlinkLogicalTableFunctionScan.scala| 7 +- .../factories/TestProcedureCatalogFactory.java | 38 .../operations/SqlNodeToCallOperationTest.java | 23 ++ .../planner/runtime/stream/sql/FunctionITCase.java | 181 +++- .../runtime/stream/sql/ProcedureITCase.java| 42 +++- 22 files changed, 840 insertions(+), 41 deletions(-) create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/annotation/ArgumentHint.java
(flink) 01/03: [FLINK-34055][table] Introduce a new annotation for named parameters
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit e1ebcaff52f423fdb54e3cb1bf8d5b3ccafc0a2f Author: Feng Jin AuthorDate: Wed Jan 17 02:17:12 2024 +0800 [FLINK-34055][table] Introduce a new annotation for named parameters --- .../flink/table/annotation/ArgumentHint.java | 69 ++ .../flink/table/annotation/FunctionHint.java | 29 + .../flink/table/annotation/ProcedureHint.java | 29 + 3 files changed, 127 insertions(+) diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/annotation/ArgumentHint.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/annotation/ArgumentHint.java new file mode 100644 index 000..10671cca6aa --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/annotation/ArgumentHint.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.table.annotation; + +import org.apache.flink.annotation.PublicEvolving; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * A hint that provides additional information about an argument. + * + * An {@code ArgumentHint} can be used to provide hints about the name, optionality, and data + * type of argument. + * + * It combines the functionality of {@link FunctionHint#argumentNames()} and {@link DataTypeHint} + * annotations to conveniently group argument-related information together in function declarations. + * + * {@code @ArgumentHint(name = "in1", type = @DataTypeHint("STRING"), isOptional = false} is an + * argument with the type String, named in1, and cannot be omitted when calling. + */ +@PublicEvolving +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE, ElementType.METHOD, ElementType.FIELD, ElementType.PARAMETER}) +public @interface ArgumentHint { + +/** + * The name of the argument. + * + * This can be used to provide a descriptive name for the argument. + */ +String name() default ""; + +/** + * Specifies whether the argument is optional or required. + * + * If set to {@code true}, the argument is considered optional.And if the user does not + * specify this parameter when calling, 'null' will be passed in. By default, an argument is + * considered required. + */ +boolean isOptional() default false; + +/** + * The data type hint for the argument. + * + * This can be used to provide additional information about the expected data type of the + * argument. The {@link DataTypeHint} annotation can be used to specify the data type explicitly + * or provide hints for the reflection-based extraction of the data type. + */ +DataTypeHint type() default @DataTypeHint(); +} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/annotation/FunctionHint.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/annotation/FunctionHint.java index 1978eecc81a..423158235ad 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/annotation/FunctionHint.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/annotation/FunctionHint.java @@ -44,6 +44,23 @@ import java.lang.annotation.Target; * {@code * // accepts (INT, STRING) and returns BOOLEAN * @FunctionHint( + * argument = [(name = "f1", @DataTypeHint("INT"), isOptional = true), + * (name = "f2", @DataTypeHint("STRING"), isOptional = true)], + * output = @DataTypeHint("BOOLEAN") + * ) + * class X extends ScalarFunction { ... } + * + * // accepts (INT, STRING...) and returns BOOLEAN + * @FunctionHint( + * argument = [(name = "f1", @DataTypeHint("INT"), isOptional = false), + * (name = "f2", @DataTypeHint("STRING"), isOptional = false)], + * i
(flink) 02/03: [FLINK-34057][table] Support named parameters for functions
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 9171194ef2647af1b55e58b98daeebabb6c84ad7 Author: Feng Jin AuthorDate: Mon Jan 22 11:41:12 2024 +0800 [FLINK-34057][table] Support named parameters for functions Co-authored-by: xuyang --- .../types/extraction/BaseMappingExtractor.java | 14 +- .../table/types/extraction/DataTypeExtractor.java | 9 +- .../table/types/extraction/DataTypeTemplate.java | 9 + .../table/types/extraction/ExtractionUtils.java| 13 +- .../extraction/FunctionSignatureTemplate.java | 6 + .../table/types/extraction/FunctionTemplate.java | 53 - .../types/extraction/TypeInferenceExtractor.java | 4 +- .../extraction/TypeInferenceExtractorTest.java | 233 - .../calcite/sql/validate/SqlValidatorImpl.java | 15 +- .../apache/calcite/sql2rel/SqlToRelConverter.java | 22 +- .../planner/calcite/RexSetSemanticsTableCall.java | 6 + .../inference/TypeInferenceOperandChecker.java | 44 +++- .../logical/FlinkLogicalTableFunctionScan.scala| 7 +- .../planner/runtime/stream/sql/FunctionITCase.java | 181 +++- 14 files changed, 585 insertions(+), 31 deletions(-) diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/BaseMappingExtractor.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/BaseMappingExtractor.java index fda3e8d6dd1..fdd12e1d795 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/BaseMappingExtractor.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/BaseMappingExtractor.java @@ -18,6 +18,7 @@ package org.apache.flink.table.types.extraction; +import org.apache.flink.table.annotation.ArgumentHint; import org.apache.flink.table.annotation.DataTypeHint; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.ValidationException; @@ -362,7 +363,18 @@ abstract class BaseMappingExtractor { Method method, int paramPos) { final Parameter parameter = method.getParameters()[paramPos]; final DataTypeHint hint = parameter.getAnnotation(DataTypeHint.class); -if (hint != null) { +final ArgumentHint argumentHint = parameter.getAnnotation(ArgumentHint.class); +if (hint != null && argumentHint != null) { +throw extractionError( +"Argument and dataType hints cannot be declared in the same parameter at position %d.", +paramPos); +} +if (argumentHint != null) { +final DataTypeTemplate template = DataTypeTemplate.fromAnnotation(argumentHint, null); +if (template.inputGroup != null) { +return Optional.of(FunctionArgumentTemplate.of(template.inputGroup)); +} +} else if (hint != null) { final DataTypeTemplate template = DataTypeTemplate.fromAnnotation(hint, null); if (template.inputGroup != null) { return Optional.of(FunctionArgumentTemplate.of(template.inputGroup)); diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/DataTypeExtractor.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/DataTypeExtractor.java index e1f0122d10e..ed05ba7a99b 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/DataTypeExtractor.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/DataTypeExtractor.java @@ -19,6 +19,7 @@ package org.apache.flink.table.types.extraction; import org.apache.flink.annotation.Internal; +import org.apache.flink.table.annotation.ArgumentHint; import org.apache.flink.table.annotation.DataTypeHint; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.dataview.DataView; @@ -140,8 +141,11 @@ public final class DataTypeExtractor { DataTypeFactory typeFactory, Class baseClass, Method method, int paramPos) { final Parameter parameter = method.getParameters()[paramPos]; final DataTypeHint hint = parameter.getAnnotation(DataTypeHint.class); +final ArgumentHint argumentHint = parameter.getAnnotation(ArgumentHint.class); final DataTypeTemplate template; -if (hint != null) { +if (argumentHint != null) { +template = DataTypeTemplate.fromAnnotation(typeFactory, argumentHint.type()); +} else if (hint != null) { template = DataTypeTemplate.fromAnnotation(typeFactory, hint); } else { template = DataTypeTemplate.fromDefaults(); @@ -261,8 +265,11 @@ public final class DataTypeExtractor {
(flink) 03/03: [FLINK-34056][table] Support named parameters for procedures
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit eb848dc0676d2c06adb3d16b3c6e51b378d4c57b Author: Feng Jin AuthorDate: Mon Jan 22 11:42:45 2024 +0800 [FLINK-34056][table] Support named parameters for procedures Co-authored-by: xuyang --- .../inference/OperatorBindingCallContext.java | 22 ++-- .../converters/SqlProcedureCallConverter.java | 13 --- .../factories/TestProcedureCatalogFactory.java | 38 .../operations/SqlNodeToCallOperationTest.java | 23 .../runtime/stream/sql/ProcedureITCase.java| 42 -- 5 files changed, 128 insertions(+), 10 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/OperatorBindingCallContext.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/OperatorBindingCallContext.java index 0bb9da4f1aa..103d37cc03e 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/OperatorBindingCallContext.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/OperatorBindingCallContext.java @@ -21,11 +21,15 @@ package org.apache.flink.table.planner.functions.inference; import org.apache.flink.annotation.Internal; import org.apache.flink.table.catalog.DataTypeFactory; import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.planner.calcite.FlinkTypeFactory; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.inference.CallContext; import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.utils.TypeConversions; import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.sql.SqlCallBinding; +import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlOperatorBinding; import javax.annotation.Nullable; @@ -63,8 +67,22 @@ public final class OperatorBindingCallContext extends AbstractSqlCallContext { new AbstractList() { @Override public DataType get(int pos) { -final LogicalType logicalType = toLogicalType(binding.getOperandType(pos)); -return fromLogicalToDataType(logicalType); +if (binding instanceof SqlCallBinding) { +SqlCallBinding sqlCallBinding = (SqlCallBinding) binding; +List operands = sqlCallBinding.operands(); +final RelDataType relDataType = +sqlCallBinding +.getValidator() +.deriveType( +sqlCallBinding.getScope(), operands.get(pos)); +final LogicalType logicalType = + FlinkTypeFactory.toLogicalType(relDataType); +return TypeConversions.fromLogicalToDataType(logicalType); +} else { +final LogicalType logicalType = +toLogicalType(binding.getOperandType(pos)); +return fromLogicalToDataType(logicalType); +} } @Override diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlProcedureCallConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlProcedureCallConverter.java index 8648937eb98..af73ccfc1e1 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlProcedureCallConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlProcedureCallConverter.java @@ -35,6 +35,7 @@ import org.apache.calcite.rex.RexLiteral; import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.ExplicitOperatorBinding; 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.SqlOperatorBinding; @@ -67,11 +68,13 @@ public class SqlProcedureCallConverter implements SqlNodeConverter { ProcedureDefinition procedureDefinition = new ProcedureDefinition(sqlProcedure.getContextResolveProcedure().getProcedure()); +SqlCallBinding sqlCallBinding = +new SqlCallBinding(context.getSqlValidator(), null, callProcedure
(flink) branch master updated (fd673a2f462 -> e345ffb453a)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from fd673a2f462 [FLINK-33974] Implement the Sink transformation depending on the new SinkV2 interfaces add e345ffb453a [FLINK-34049][table] Refactor classes related to window TVF aggregation to prepare for non-aligned windows (#24068) No new revisions were added by this update. Summary of changes: .../PythonStreamGroupWindowAggregateOperator.java | 14 +- ...PythonGroupWindowAggregateFunctionOperator.java | 12 +- ...ghPythonStreamGroupWindowAggregateOperator.java | 4 +- ...thonStreamGroupWindowAggregateOperatorTest.java | 2 +- ...onGroupWindowAggregateFunctionOperatorTest.java | 10 +- .../logical/SliceAttachedWindowingStrategy.java| 2 +- .../exec/common/CommonExecWindowTableFunction.java | 6 +- .../stream/StreamExecGlobalWindowAggregate.java| 2 +- .../stream/StreamExecGroupWindowAggregate.java | 4 +- .../stream/StreamExecLocalWindowAggregate.java | 2 +- .../StreamExecPythonGroupWindowAggregate.java | 44 +++--- .../exec/stream/StreamExecWindowAggregate.java | 4 +- .../exec/stream/StreamExecWindowAggregateBase.java | 4 +- .../plan/utils/WindowTableFunctionUtil.java| 10 +- .../codegen/agg/AggsHandlerCodeGenerator.scala | 4 +- .../planner/plan/utils/WindowEmitStrategy.scala| 2 +- .../window/LocalSlicingWindowAggOperator.java | 4 +- .../window/SlicingWindowAggOperatorBuilder.java| 14 +- .../window/buffers/RecordsWindowBuffer.java| 6 +- .../aggregate/window/buffers/WindowBuffer.java | 4 +- .../aggregate/window/combines/AggCombiner.java | 10 +- .../window/combines/GlobalAggCombiner.java | 10 +- .../window/combines/LocalAggCombiner.java | 2 +- ...r.java => AbstractSliceWindowAggProcessor.java} | 133 +++- .../processors/AbstractWindowAggProcessor.java | 173 + .../processors/SliceSharedWindowAggProcessor.java | 9 +- .../SliceUnsharedWindowAggProcessor.java | 4 +- .../RowTimeWindowDeduplicateOperatorBuilder.java | 6 +- .../RowTimeDeduplicateRecordsCombiner.java | 10 +- .../RowTimeWindowDeduplicateProcessor.java | 11 +- .../operators/join/window/WindowJoinOperator.java | 9 +- .../rank/window/WindowRankOperatorBuilder.java | 6 +- .../rank/window/combines/TopNRecordsCombiner.java | 10 +- .../window/processors/WindowRankProcessor.java | 11 +- .../runtime/operators/window/MergeCallback.java| 43 + .../assigners/CountSlidingWindowAssigner.java | 10 +- .../assigners/CountTumblingWindowAssigner.java | 10 +- .../assigners/CumulativeWindowAssigner.java| 6 +- .../assigners/GroupWindowAssigner.java}| 15 +- .../assigners/InternalTimeWindowAssigner.java | 2 +- .../assigners/MergingWindowAssigner.java | 24 +-- .../assigners/PanedWindowAssigner.java | 4 +- .../assigners/SessionWindowAssigner.java | 17 +- .../assigners/SlidingWindowAssigner.java | 6 +- .../assigners/TumblingWindowAssigner.java | 16 +- .../internal/GeneralWindowProcessFunction.java | 12 +- .../internal/InternalWindowProcessFunction.java| 10 +- .../internal/MergingWindowProcessFunction.java | 4 +- .../internal/MergingWindowSet.java | 7 +- .../internal/PanedWindowProcessFunction.java | 4 +- .../operator}/AggregateWindowOperator.java | 17 +- .../operator}/TableAggregateWindowOperator.java| 17 +- .../{ => groupwindow/operator}/WindowOperator.java | 35 +++-- .../operator}/WindowOperatorBuilder.java | 34 ++-- .../triggers/ElementTriggers.java | 2 +- .../triggers/EventTimeTriggers.java| 2 +- .../triggers/ProcessingTimeTriggers.java | 2 +- .../window/{ => groupwindow}/triggers/Trigger.java | 8 +- .../{ => groupwindow}/triggers/WindowTrigger.java | 7 +- .../window/{ => tvf}/combines/RecordsCombiner.java | 6 +- .../{slicing => tvf/common}/ClockService.java | 2 +- .../window/tvf/common/WindowAssigner.java | 41 + .../common/WindowOperatorBase.java}| 74 - .../common/WindowProcessor.java} | 8 +- .../common}/WindowTimerService.java| 2 +- .../common/WindowTimerServiceBase.java}| 27 +--- .../operator}/WindowTableFunctionOperator.java | 11 +- .../window/{ => tvf}/slicing/SliceAssigner.java| 14 +- .../window/{ => tvf}/slicing/SliceAssigners.java | 13 +- .../{ => tvf}/slicing/SliceSharedAssigner.java | 25 +-- .../{ => tvf}/slicing/SliceUnsharedAssigner.java | 2 +- .../window/tvf/slicing/SlicingWindowOperator.java | 63 .../slici
(flink) branch master updated: [FLINK-33928][table-planner] Should not throw exception while creating view with specify field names (#24096)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new 82fcdfe5634 [FLINK-33928][table-planner] Should not throw exception while creating view with specify field names (#24096) 82fcdfe5634 is described below commit 82fcdfe5634fb82d3ab4a183818d852119dc68a9 Author: yunhong <337361...@qq.com> AuthorDate: Fri Jan 19 11:16:25 2024 +0800 [FLINK-33928][table-planner] Should not throw exception while creating view with specify field names (#24096) Co-authored-by: zhengyunhong.zyh --- .../operations/converters/SqlNodeConvertUtils.java | 23 -- .../operations/SqlDdlToOperationConverterTest.java | 13 2 files changed, 30 insertions(+), 6 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlNodeConvertUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlNodeConvertUtils.java index 43e3937392e..991bbf744b5 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlNodeConvertUtils.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlNodeConvertUtils.java @@ -81,7 +81,7 @@ class SqlNodeConvertUtils { // index to identify the duplicate names. SqlValidatorNamespace validatedNamespace = context.getSqlValidator().getNamespace(validateQuery); -validateDuplicatedColumnNames(query, validatedNamespace); +validateDuplicatedColumnNames(query, viewFields, validatedNamespace); // The LATERAL operator was eliminated during sql validation, thus the unparsed SQL // does not contain LATERAL which is problematic, @@ -142,12 +142,23 @@ class SqlNodeConvertUtils { } private static void validateDuplicatedColumnNames( -SqlNode query, SqlValidatorNamespace namespace) { +SqlNode query, List viewFields, SqlValidatorNamespace namespace) { + +// If view fields is not empty, means the view column list is specified by user use syntax +// 'CREATE VIEW viewName(x,x,x) AS SELECT x,x,x FROM table'. For this syntax, we need +// validate whether the column name in the view column list is unique. +List columnNameList; +if (!viewFields.isEmpty()) { +columnNameList = + viewFields.stream().map(SqlNode::toString).collect(Collectors.toList()); +} else { +Objects.requireNonNull(namespace); +columnNameList = namespace.getType().getFieldNames(); +} + Map nameToPos = new HashMap<>(); -for (int i = 0; -i < Objects.requireNonNull(namespace).getType().getFieldList().size(); -i++) { -String columnName = namespace.getType().getFieldList().get(i).getName(); +for (int i = 0; i < columnNameList.size(); i++) { +String columnName = columnNameList.get(i); if (nameToPos.containsKey(columnName)) { SqlSelect select = extractSelect(query); // Can not get the origin schema. diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlDdlToOperationConverterTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlDdlToOperationConverterTest.java index 1de2d72583a..c88140e5780 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlDdlToOperationConverterTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlDdlToOperationConverterTest.java @@ -2229,6 +2229,19 @@ public class SqlDdlToOperationConverterTest extends SqlNodeToOperationConversion catalogManager.createTable( catalogTable, ObjectIdentifier.of("builtin", "default", "id_table"), false); +Operation operation = +parse("CREATE VIEW id_view(a, b) AS SELECT id, uid AS id FROM id_table"); +assertThat(operation).isInstanceOf(CreateViewOperation.class); + +assertThatThrownBy( +() -> +parse( +"CREATE VIEW id_view(a, a) AS SELECT id, uid AS id FROM id_table")) +.satisfies( +FlinkAssertions.anyCauseMatches( +SqlValidateException.class, +"A column with the same name `a` has been defined at line 1, column 37.")); + assertThatThrownB
(flink) branch master updated (d92ab390ff0 -> 488d60a1d39)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from d92ab390ff0 [FLINK-27756] Refactor Async Sink send gauge test add 488d60a1d39 [FLINK-34066][table-planner] Fix LagFunction throw NPE when input argument are not null (#24075) No new revisions were added by this update. Summary of changes: .../flink/table/planner/runtime/stream/sql/AggregateITCase.scala | 5 +++-- .../flink/table/runtime/functions/aggregate/LagAggFunction.java | 3 ++- 2 files changed, 5 insertions(+), 3 deletions(-)
(flink) branch master updated: [FLINK-33490][table-planner] Validate the column name conflicts in view when creating view (#23760)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new a4fe01cb9d6 [FLINK-33490][table-planner] Validate the column name conflicts in view when creating view (#23760) a4fe01cb9d6 is described below commit a4fe01cb9d678b293107b0a6278fec1a749913cc Author: Xuyang AuthorDate: Thu Dec 21 11:13:00 2023 +0800 [FLINK-33490][table-planner] Validate the column name conflicts in view when creating view (#23760) --- .../operations/converters/SqlNodeConvertUtils.java | 69 ++ .../operations/SqlDdlToOperationConverterTest.java | 52 2 files changed, 121 insertions(+) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlNodeConvertUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlNodeConvertUtils.java index 06cdceea9ad..43e3937392e 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlNodeConvertUtils.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlNodeConvertUtils.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.operations.converters; import org.apache.flink.sql.parser.ddl.SqlAlterView; +import org.apache.flink.sql.parser.error.SqlValidateException; import org.apache.flink.table.api.Schema; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.CatalogBaseTable; @@ -32,10 +33,20 @@ import org.apache.flink.table.planner.operations.PlannerQueryOperation; import org.apache.flink.table.planner.operations.converters.SqlNodeConverter.ConvertContext; import org.apache.calcite.rel.RelRoot; +import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlSelect; +import org.apache.calcite.sql.SqlSetOperator; +import org.apache.calcite.sql.SqlWith; +import org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.calcite.sql.validate.SqlValidatorNamespace; +import javax.annotation.Nullable; + +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.stream.Collectors; @@ -64,6 +75,14 @@ class SqlNodeConvertUtils { // This bug is fixed in CALCITE-3877 of Calcite 1.23.0. String originalQuery = context.toQuotedSqlString(query); SqlNode validateQuery = context.getSqlValidator().validate(query); + +// Check name is unique. +// Don't rely on the calcite because if the field names are duplicate, calcite will add +// index to identify the duplicate names. +SqlValidatorNamespace validatedNamespace = +context.getSqlValidator().getNamespace(validateQuery); +validateDuplicatedColumnNames(query, validatedNamespace); + // The LATERAL operator was eliminated during sql validation, thus the unparsed SQL // does not contain LATERAL which is problematic, // the issue was resolved in CALCITE-4077 @@ -121,4 +140,54 @@ class SqlNodeConvertUtils { } return (CatalogView) baseTable; } + +private static void validateDuplicatedColumnNames( +SqlNode query, SqlValidatorNamespace namespace) { +Map nameToPos = new HashMap<>(); +for (int i = 0; +i < Objects.requireNonNull(namespace).getType().getFieldList().size(); +i++) { +String columnName = namespace.getType().getFieldList().get(i).getName(); +if (nameToPos.containsKey(columnName)) { +SqlSelect select = extractSelect(query); +// Can not get the origin schema. +if (select == null) { +throw new ValidationException( +String.format( +"SQL validation failed. Column `%s` has been specified.", +columnName)); +} +SqlParserPos errorPos = select.getSelectList().get(i).getParserPosition(); +String msg = +String.format( +"A column with the same name `%s` has been defined at %s.", +columnName, +select.getSelectList() +.get(nameToPos.get(columnName)) +.getParserPosition()); +throw new ValidationException( +"SQL validation failed. " + msg, new SqlValidateException(errorPos, msg)); +
(flink) branch pull/23760/head deleted (was d68e775022d)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch pull/23760/head in repository https://gitbox.apache.org/repos/asf/flink.git was d68e775022d [FLINK-33490][table-planner] Validate the name conflicts when creating view This change permanently discards the following revisions: discard d68e775022d [FLINK-33490][table-planner] Validate the name conflicts when creating view
(flink) branch pull/23760/head created (now d68e775022d)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch pull/23760/head in repository https://gitbox.apache.org/repos/asf/flink.git at d68e775022d [FLINK-33490][table-planner] Validate the name conflicts when creating view This branch includes the following new commits: new d68e775022d [FLINK-33490][table-planner] Validate the name conflicts when creating view The 1 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference.
(flink) 01/01: [FLINK-33490][table-planner] Validate the name conflicts when creating view
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch pull/23760/head in repository https://gitbox.apache.org/repos/asf/flink.git commit d68e775022d9c8f52e8c61b0b7c34bbf2ba30baa Author: xuyang AuthorDate: Mon Nov 20 19:43:39 2023 +0800 [FLINK-33490][table-planner] Validate the name conflicts when creating view --- .../operations/converters/SqlNodeConvertUtils.java | 69 ++ .../operations/SqlDdlToOperationConverterTest.java | 52 2 files changed, 121 insertions(+) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlNodeConvertUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlNodeConvertUtils.java index 28d030af707..727b136123f 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlNodeConvertUtils.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlNodeConvertUtils.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.operations.converters; import org.apache.flink.sql.parser.ddl.SqlAlterView; +import org.apache.flink.sql.parser.error.SqlValidateException; import org.apache.flink.table.api.Schema; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.CatalogBaseTable; @@ -32,10 +33,20 @@ import org.apache.flink.table.planner.operations.PlannerQueryOperation; import org.apache.flink.table.planner.operations.converters.SqlNodeConverter.ConvertContext; import org.apache.calcite.rel.RelRoot; +import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlSelect; +import org.apache.calcite.sql.SqlSetOperator; +import org.apache.calcite.sql.SqlWith; +import org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.calcite.sql.validate.SqlValidatorNamespace; +import javax.annotation.Nullable; + +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.stream.Collectors; @@ -63,6 +74,14 @@ class SqlNodeConvertUtils { // This bug is fixed in CALCITE-3877 of Calcite 1.23.0. String originalQuery = context.toQuotedSqlString(query); SqlNode validateQuery = context.getSqlValidator().validate(query); + +// Check name is unique. +// Don't rely on the calcite because if the field names are duplicate, calcite will add +// index to identify the duplicate names. +SqlValidatorNamespace validatedNamespace = +context.getSqlValidator().getNamespace(validateQuery); +validateDuplicatedColumnNames(query, validatedNamespace); + // The LATERAL operator was eliminated during sql validation, thus the unparsed SQL // does not contain LATERAL which is problematic, // the issue was resolved in CALCITE-4077 @@ -120,4 +139,54 @@ class SqlNodeConvertUtils { } return (CatalogView) baseTable; } + +private static void validateDuplicatedColumnNames( +SqlNode query, SqlValidatorNamespace namespace) { +Map nameToPos = new HashMap<>(); +for (int i = 0; +i < Objects.requireNonNull(namespace).getType().getFieldList().size(); +i++) { +String columnName = namespace.getType().getFieldList().get(i).getName(); +if (nameToPos.containsKey(columnName)) { +SqlSelect select = extractSelect(query); +// Can not get the origin schema. +if (select == null) { +throw new ValidationException( +String.format( +"SQL validation failed. Column `%s` has been specified.", +columnName)); +} +SqlParserPos errorPos = select.getSelectList().get(i).getParserPosition(); +String msg = +String.format( +"A column with the same name `%s` has been defined at %s.", +columnName, +select.getSelectList() +.get(nameToPos.get(columnName)) +.getParserPosition()); +throw new ValidationException( +"SQL validation failed. " + msg, new SqlValidateException(errorPos, msg)); +} +nameToPos.put(columnName, i); +} +} + +private static @Nullable SqlSelect extractSelect(SqlNode query) { +if (query instanceof SqlSelect) { +return (SqlSelect) query; +
(flink) branch master updated: [hotfix][test] Migrate JsonRowDeserializationSchemaTest/JsonRowSerializationSchemaTest to Junit5 and Assertj (#23882)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new e454fc07192 [hotfix][test] Migrate JsonRowDeserializationSchemaTest/JsonRowSerializationSchemaTest to Junit5 and Assertj (#23882) e454fc07192 is described below commit e454fc0719276d2d54590989b754e88b9adb4455 Author: yunhong <337361...@qq.com> AuthorDate: Tue Dec 12 10:19:08 2023 +0800 [hotfix][test] Migrate JsonRowDeserializationSchemaTest/JsonRowSerializationSchemaTest to Junit5 and Assertj (#23882) Co-authored-by: zhengyunhong.zyh --- .../json/JsonRowDeserializationSchemaTest.java | 99 -- .../json/JsonRowSerializationSchemaTest.java | 46 +- 2 files changed, 78 insertions(+), 67 deletions(-) diff --git a/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java b/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java index 81e370c0fd3..4712768f291 100644 --- a/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java +++ b/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java @@ -26,10 +26,7 @@ import org.apache.flink.util.jackson.JacksonMapperFactory; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; +import org.junit.jupiter.api.Test; import javax.annotation.Nullable; @@ -44,19 +41,14 @@ import java.util.Map; import java.util.concurrent.ThreadLocalRandom; import static org.apache.flink.formats.utils.DeserializationSchemaMatcher.whenDeserializedWith; -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.instanceOf; -import static org.junit.Assert.assertThat; -import static org.junit.internal.matchers.ThrowableCauseMatcher.hasCause; -import static org.junit.internal.matchers.ThrowableMessageMatcher.hasMessage; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for the {@link JsonRowDeserializationSchema}. */ public class JsonRowDeserializationSchemaTest { private static final ObjectMapper OBJECT_MAPPER = JacksonMapperFactory.createObjectMapper(); -@Rule public ExpectedException thrown = ExpectedException.none(); - /** Tests simple deserialization using type information. */ @Test public void testTypeInfoDeserialization() throws Exception { @@ -135,7 +127,11 @@ public class JsonRowDeserializationSchemaTest { row.setField(9, map); row.setField(10, nestedMap); -assertThat(serializedJson, whenDeserializedWith(deserializationSchema).equalsTo(row)); +assertThat( +whenDeserializedWith(deserializationSchema) +.equalsTo(row) +.matches(serializedJson)) +.isTrue(); } @Test @@ -205,7 +201,11 @@ public class JsonRowDeserializationSchemaTest { nestedRow.setField(1, BigDecimal.valueOf(12)); expected.setField(9, nestedRow); -assertThat(serializedJson, whenDeserializedWith(deserializationSchema).equalsTo(expected)); +assertThat( +whenDeserializedWith(deserializationSchema) +.equalsTo(expected) +.matches(serializedJson)) +.isTrue(); } /** Tests deserialization with non-existing field name. */ @@ -223,45 +223,55 @@ public class JsonRowDeserializationSchemaTest { new JsonRowDeserializationSchema.Builder(rowTypeInformation).build(); Row row = new Row(1); -assertThat(serializedJson, whenDeserializedWith(deserializationSchema).equalsTo(row)); +assertThat( +whenDeserializedWith(deserializationSchema) +.equalsTo(row) +.matches(serializedJson)) +.isTrue(); deserializationSchema = new JsonRowDeserializationSchema.Builder(rowTypeInformation) .failOnMissingField() .build(); - -assertThat( -serializedJson, -whenDeserializedWith(deserializationSchema) - .failsWithException(hasCause(instanceOf(IllegalStateException.class; +final JsonRowDeserializationSchema errorDs = deserializationSchema; +assertThatTh
(flink) branch master updated: [FLINK-33672][table-runtime] Use MapState.entries() instead of keys() and get() in over window (#23855)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new 080119cca53 [FLINK-33672][table-runtime] Use MapState.entries() instead of keys() and get() in over window (#23855) 080119cca53 is described below commit 080119cca53d9890257982b6a74a7d6f913253c2 Author: Zakelly AuthorDate: Tue Dec 12 10:14:04 2023 +0800 [FLINK-33672][table-runtime] Use MapState.entries() instead of keys() and get() in over window (#23855) --- .../operators/over/ProcTimeRangeBoundedPrecedingFunction.java | 8 +--- .../operators/over/RowTimeRangeBoundedPrecedingFunction.java | 10 ++ .../operators/over/RowTimeRowsBoundedPrecedingFunction.java| 6 -- 3 files changed, 15 insertions(+), 9 deletions(-) diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeRangeBoundedPrecedingFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeRangeBoundedPrecedingFunction.java index 39405ec8a05..15c5f0e4141 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeRangeBoundedPrecedingFunction.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeRangeBoundedPrecedingFunction.java @@ -43,6 +43,7 @@ import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.Iterator; import java.util.List; +import java.util.Map; /** * Process Function used for the aggregate in bounded proc-time OVER window. @@ -197,13 +198,14 @@ public class ProcTimeRangeBoundedPrecedingFunction // when we find timestamps that are out of interest, we retrieve corresponding elements // and eliminate them. Multiple elements could have been received at the same timestamp // the removal of old elements happens only once per proctime as onTimer is called only once -Iterator iter = inputState.keys().iterator(); +Iterator>> iter = inputState.entries().iterator(); List markToRemove = new ArrayList(); while (iter.hasNext()) { -Long elementKey = iter.next(); +Map.Entry> element = iter.next(); +Long elementKey = element.getKey(); if (elementKey < limit) { // element key outside of window. Retract values -List elementsRemove = inputState.get(elementKey); +List elementsRemove = element.getValue(); if (elementsRemove != null) { int iRemove = 0; while (iRemove < elementsRemove.size()) { diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRangeBoundedPrecedingFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRangeBoundedPrecedingFunction.java index bf5a3d5ca7f..285b8de6f1b 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRangeBoundedPrecedingFunction.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRangeBoundedPrecedingFunction.java @@ -44,6 +44,7 @@ import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.Iterator; import java.util.List; +import java.util.Map; /** * Process Function for RANGE clause event-time bounded OVER window. @@ -227,12 +228,13 @@ public class RowTimeRangeBoundedPrecedingFunction List retractTsList = new ArrayList(); // do retraction -Iterator dataTimestampIt = inputState.keys().iterator(); -while (dataTimestampIt.hasNext()) { -Long dataTs = dataTimestampIt.next(); +Iterator>> iter = inputState.entries().iterator(); +while (iter.hasNext()) { +Map.Entry> data = iter.next(); +Long dataTs = data.getKey(); Long offset = timestamp - dataTs; if (offset > precedingOffset) { -List retractDataList = inputState.get(dataTs); +List retractDataList = data.getValue(); if (retractDataList != null) { dataListIndex = 0; while (dataListIndex < retractDataList.size()) { diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRowsBoundedPrecedingFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRowsBoundedPrecedingFunction.java index 74ffb428c2d..df897eef9a0 100644 --- a/flink-table/flink-table-runtime/
(flink) branch master updated: [FLINK-33691][planner] Support agg push down for 'count(*)/count(1)/count(column not null)' (#23828)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new 3060ccd49cc [FLINK-33691][planner] Support agg push down for 'count(*)/count(1)/count(column not null)' (#23828) 3060ccd49cc is described below commit 3060ccd49cc8d19634b431dbf0f09ac875d0d422 Author: yunhong <337361...@qq.com> AuthorDate: Wed Dec 6 17:35:52 2023 +0800 [FLINK-33691][planner] Support agg push down for 'count(*)/count(1)/count(column not null)' (#23828) Co-authored-by: zhengyunhong.zyh --- .../batch/PushLocalAggIntoScanRuleBase.java| 36 +++-- .../batch/PushLocalHashAggIntoScanRule.java| 4 +- .../PushLocalHashAggWithCalcIntoScanRule.java | 10 +- .../batch/PushLocalSortAggIntoScanRule.java| 4 +- .../PushLocalSortAggWithCalcIntoScanRule.java | 10 +- ...ushLocalSortAggWithSortAndCalcIntoScanRule.java | 10 +- .../PushLocalSortAggWithSortIntoScanRule.java | 4 +- .../PushLocalAggIntoTableSourceScanRuleTest.java | 22 ++- .../sql/agg/LocalAggregatePushDownITCase.java | 42 +- .../plan/batch/sql/agg/HashAggregateTest.xml | 134 - .../plan/batch/sql/agg/SortAggregateTest.xml | 132 - .../PushLocalAggIntoTableSourceScanRuleTest.xml| 164 +++-- 12 files changed, 350 insertions(+), 222 deletions(-) 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 81bdaf68767..c576e5b684d 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 @@ -45,8 +45,11 @@ import org.apache.calcite.rel.core.AggregateCall; import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexNode; import org.apache.calcite.rex.RexProgram; +import org.apache.calcite.sql.SqlKind; import org.apache.commons.lang3.ArrayUtils; +import javax.annotation.Nullable; + import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -76,7 +79,8 @@ public abstract class PushLocalAggIntoScanRuleBase extends RelOptRule { protected boolean canPushDown( RelOptRuleCall call, BatchPhysicalGroupAggregateBase aggregate, -BatchPhysicalTableSourceScan tableSourceScan) { +BatchPhysicalTableSourceScan tableSourceScan, +@Nullable BatchPhysicalCalc calc) { TableConfig tableConfig = ShortcutUtils.unwrapContext(call.getPlanner()).getTableConfig(); if (!tableConfig.get( OptimizerConfigOptions.TABLE_OPTIMIZER_SOURCE_AGGREGATE_PUSHDOWN_ENABLED)) { @@ -86,9 +90,24 @@ public abstract class PushLocalAggIntoScanRuleBase extends RelOptRule { if (aggregate.isFinal() || aggregate.getAggCallList().isEmpty()) { return false; } + List aggCallList = JavaScalaConversionUtil.toJava(aggregate.getAggCallList()); for (AggregateCall aggCall : aggCallList) { +// We should try best to push down count(*) and count(n) even if it was optimized to a +// calc as '0 AS $f0' to reduce read cost. +if (aggCall.getAggregation().kind != SqlKind.COUNT && calc != null) { +if (!isInputRefOnly(calc) || !isProjectionNotPushedDown(tableSourceScan)) { +return false; +} +} else if (aggCall.getAggregation().kind == SqlKind.COUNT && calc != null) { +// count(*) push down shouldn't have filter condition upon source. For example +// 'select count(*) from source where a > 10' cannot push down now. +if (calc.getProgram().getCondition() != null) { +return false; +} +} + if (aggCall.isDistinct() || aggCall.isApproximate() || aggCall.getArgList().size() > 1 @@ -105,24 +124,23 @@ public abstract class PushLocalAggIntoScanRuleBase extends RelOptRule { .noneMatch(spec -> spec instanceof AggregatePushDownSpec); } -protected void pushLocalAggregateIntoScan( -RelOptRuleCall call, -BatchPhysicalGroupAggregateBase localAgg, -BatchPhysicalTableSourceScan oldScan) { -pushLocalAggregateIntoScan(call, local
(flink) branch master updated: [FLINK-33489][table-planner] forbid generating partial-final agg with LISTAGG to avoid wrong result
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new 7e30e5e9fcd [FLINK-33489][table-planner] forbid generating partial-final agg with LISTAGG to avoid wrong result 7e30e5e9fcd is described below commit 7e30e5e9fcd51382f48d48c9848bb8df14293e22 Author: xuyang AuthorDate: Thu Nov 9 12:28:14 2023 +0800 [FLINK-33489][table-planner] forbid generating partial-final agg with LISTAGG to avoid wrong result --- .../table/planner/plan/utils/AggregateUtil.scala | 5 +- .../plan/stream/sql/agg/DistinctAggregateTest.xml | 86 ++ .../stream/sql/agg/IncrementalAggregateTest.xml| 22 ++ .../stream/sql/agg/DistinctAggregateTest.scala | 5 ++ .../runtime/stream/sql/SplitAggregateITCase.scala | 31 5 files changed, 147 insertions(+), 2 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala index ef25844477f..73c5f2a09da 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala @@ -953,9 +953,10 @@ object AggregateUtil extends Enumeration { aggCall.getAggregation match { case _: SqlCountAggFunction | _: SqlAvgAggFunction | _: SqlMinMaxAggFunction | _: SqlSumAggFunction | _: SqlSumEmptyIsZeroAggFunction | - _: SqlSingleValueAggFunction | _: SqlListAggFunction => + _: SqlSingleValueAggFunction => true - case _: SqlFirstLastValueAggFunction => aggCall.getArgList.size() == 1 + case _: SqlFirstLastValueAggFunction | _: SqlListAggFunction => +aggCall.getArgList.size() == 1 case _ => false } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.xml index cf1f3660adb..6ea406b774f 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.xml @@ -287,6 +287,92 @@ GroupAggregate(groupBy=[b], partialFinalType=[FINAL], select=[b, FIRST_VALUE_RET +- Calc(select=[a, b, MOD(HASH_CODE(b), 1024) AS $f2], changelogMode=[I]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime], changelogMode=[I]) +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.xml index 3962d19ab57..8585541ae0c 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.xml @@ -85,6 +85,28 @@ GroupAggregate(groupBy=[b], partialFinalType=[FINAL], select=[b, FIRST_VALUE_RET +- Calc(select=[a, b, MOD(HASH_CODE(b), 1024) AS $f2], changelogMode=[I]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime], changelogMode=[I]) +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]) +]]> + + + + + + + + + + + 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.sc
[flink] branch master updated (935188f06a8 -> a29a320187a)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 935188f06a8 [FLINK-33236][config] Deprecate the unused high-availability.zookeeper.path.running-registry option (#23506) add a29a320187a [FLINK-33226][table] Forbid to drop current database No new revisions were added by this update. Summary of changes: .../flink/connectors/hive/HiveRunnerITCase.java| 800 +++-- .../flink/connectors/hive/HiveTableSinkITCase.java | 707 +- .../hive/HiveTableSpeculativeSinkITCase.java | 160 +++-- .../connectors/hive/TableEnvExecutorUtil.java | 70 ++ .../hive/TableEnvHiveConnectorITCase.java | 682 +- .../table/catalog/hive/HiveCatalogITCase.java | 90 +-- .../src/test/resources/sql/catalog_database.q | 5 +- .../src/test/resources/sql/catalog_database.q | 9 +- .../apache/flink/table/catalog/CatalogManager.java | 5 + .../flink/table/catalog/CatalogManagerTest.java| 74 +- .../table/planner/catalog/CatalogTableITCase.scala | 7 + 11 files changed, 1386 insertions(+), 1223 deletions(-) create mode 100644 flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvExecutorUtil.java
[flink] 01/03: [FLINK-33000][sql-gateway] SqlGatewayServiceITCase should utilize TestExecutorExtension instead of using a ThreadFactory
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit de6c66ea805760f1550ae1fa348630edd9f17256 Author: Jiabao Sun AuthorDate: Thu Sep 21 15:30:45 2023 +0800 [FLINK-33000][sql-gateway] SqlGatewayServiceITCase should utilize TestExecutorExtension instead of using a ThreadFactory --- .../gateway/service/SqlGatewayServiceITCase.java | 59 +- 1 file changed, 34 insertions(+), 25 deletions(-) diff --git a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/SqlGatewayServiceITCase.java b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/SqlGatewayServiceITCase.java index b74db280f06..3e62cdf90d1 100644 --- a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/SqlGatewayServiceITCase.java +++ b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/SqlGatewayServiceITCase.java @@ -66,6 +66,7 @@ import org.apache.flink.table.planner.utils.TableFunc0; import org.apache.flink.test.junit5.InjectClusterClient; import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.test.util.TestUtils; +import org.apache.flink.testutils.executor.TestExecutorExtension; import org.apache.flink.util.CollectionUtil; import org.apache.flink.util.UserClassLoaderJarTestUtils; import org.apache.flink.util.concurrent.ExecutorThreadFactory; @@ -96,9 +97,10 @@ import java.util.Random; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.FutureTask; import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicReference; import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; @@ -125,7 +127,7 @@ public class SqlGatewayServiceITCase { @RegisterExtension @Order(1) -public static final MiniClusterExtension MINI_CLUSTER = +static final MiniClusterExtension MINI_CLUSTER = new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(2) @@ -133,9 +135,19 @@ public class SqlGatewayServiceITCase { @RegisterExtension @Order(2) -public static final SqlGatewayServiceExtension SQL_GATEWAY_SERVICE_EXTENSION = +static final SqlGatewayServiceExtension SQL_GATEWAY_SERVICE_EXTENSION = new SqlGatewayServiceExtension(MINI_CLUSTER::getClientConfiguration); +@RegisterExtension +@Order(3) +static final TestExecutorExtension EXECUTOR_EXTENSION = +new TestExecutorExtension<>( +() -> +Executors.newCachedThreadPool( +new ExecutorThreadFactory( +"SqlGatewayService Test Pool", +IgnoreExceptionHandler.INSTANCE))); + private static SessionManagerImpl sessionManager; private static SqlGatewayServiceImpl service; @@ -143,9 +155,6 @@ public class SqlGatewayServiceITCase { SessionEnvironment.newBuilder() .setSessionEndpointVersion(MockedEndpointVersion.V1) .build(); -private final ThreadFactory threadFactory = -new ExecutorThreadFactory( -"SqlGatewayService Test Pool", IgnoreExceptionHandler.INSTANCE); @BeforeAll static void setUp() { @@ -773,12 +782,10 @@ public class SqlGatewayServiceITCase { service.getSession(sessionHandle) .getOperationManager() .getOperation(operationHandle)); -threadFactory -.newThread(() -> service.cancelOperation(sessionHandle, operationHandle)) -.start(); -threadFactory -.newThread(() -> service.closeOperation(sessionHandle, operationHandle)) -.start(); + +ExecutorService executor = EXECUTOR_EXTENSION.getExecutor(); +executor.submit(() -> service.cancelOperation(sessionHandle, operationHandle)); +executor.submit(() -> service.closeOperation(sessionHandle, operationHandle)); } CommonTestUtils.waitUtil( @@ -800,16 +807,16 @@ public class SqlGatewayServiceITCase { int submitThreadsNum = 100; CountDownLatch latch = new CountDownLatch(submitThreadsNum); for (int i = 0; i < submitThreadsNum; i++) { -threadFactory -.newThread( +EXECUTOR_EXTENSION +
[flink] 02/03: [FLINK-33000][sql-gateway] OperationManagerTest should utilize TestExecutorExtension instead of using a ThreadFactory
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 73717520cc63df8bd08fd008ac004659b210cfd1 Author: Jiabao Sun AuthorDate: Thu Sep 21 17:43:38 2023 +0800 [FLINK-33000][sql-gateway] OperationManagerTest should utilize TestExecutorExtension instead of using a ThreadFactory --- .../service/operation/OperationManagerTest.java| 43 +++--- 1 file changed, 22 insertions(+), 21 deletions(-) diff --git a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/operation/OperationManagerTest.java b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/operation/OperationManagerTest.java index d734ad8afa3..fec988939c6 100644 --- a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/operation/OperationManagerTest.java +++ b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/operation/OperationManagerTest.java @@ -30,22 +30,22 @@ import org.apache.flink.table.gateway.api.operation.OperationStatus; import org.apache.flink.table.gateway.api.results.ResultSet; import org.apache.flink.table.gateway.api.results.ResultSetImpl; import org.apache.flink.table.gateway.api.utils.SqlGatewayException; -import org.apache.flink.table.gateway.api.utils.ThreadUtils; import org.apache.flink.table.gateway.service.utils.IgnoreExceptionHandler; import org.apache.flink.table.gateway.service.utils.SqlCancelException; import org.apache.flink.table.gateway.service.utils.SqlExecutionException; +import org.apache.flink.testutils.executor.TestExecutorExtension; import org.apache.flink.util.concurrent.ExecutorThreadFactory; -import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; import java.time.Duration; import java.util.Collections; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; -import java.util.concurrent.ThreadFactory; +import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicReference; import static org.apache.flink.table.api.internal.StaticResultProvider.SIMPLE_ROW_DATA_TO_STRING_CONVERTER; @@ -56,19 +56,21 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Test for {@link OperationManager}. */ class OperationManagerTest { -private static final ExecutorService EXECUTOR_SERVICE = -ThreadUtils.newThreadPool(5, 500, 60_, "operation-manager-test"); - private static OperationManager operationManager; private static ResultSet defaultResultSet; -private final ThreadFactory threadFactory = -new ExecutorThreadFactory( -"SqlGatewayService Test Pool", IgnoreExceptionHandler.INSTANCE); +@RegisterExtension +private static final TestExecutorExtension EXECUTOR_EXTENSION = +new TestExecutorExtension<>( +() -> +Executors.newCachedThreadPool( +new ExecutorThreadFactory( +"SqlGatewayService Test Pool", +IgnoreExceptionHandler.INSTANCE))); @BeforeEach void setUp() { -operationManager = new OperationManager(EXECUTOR_SERVICE); +operationManager = new OperationManager(EXECUTOR_EXTENSION.getExecutor()); defaultResultSet = new ResultSetImpl( PAYLOAD, @@ -86,11 +88,6 @@ class OperationManagerTest { operationManager.close(); } -@AfterAll -static void cleanUp() { -EXECUTOR_SERVICE.shutdown(); -} - @Test void testRunOperationAsynchronously() throws Exception { OperationHandle operationHandle = operationManager.submitOperation(() -> defaultResultSet); @@ -127,7 +124,9 @@ class OperationManagerTest { return defaultResultSet; }); -threadFactory.newThread(() -> operationManager.cancelOperation(operationHandle)).start(); +EXECUTOR_EXTENSION +.getExecutor() +.submit(() -> operationManager.cancelOperation(operationHandle)); operationManager.awaitOperationTermination(operationHandle); assertThat(operationManager.getOperationInfo(operationHandle).getStatus()) @@ -164,8 +163,9 @@ class OperationManagerTest { void testCloseUninterruptedOperation() throws Exception { AtomicReference isRunning = new AtomicReference<>(false); for (int i = 0; i < 10; i++) { -threadFactory -.newThread( +
[flink] branch master updated (5525d85e70b -> 8e68b40d5cf)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 5525d85e70b [FLINK-33144][datastream]Deprecate Iteration API in DataStream new de6c66ea805 [FLINK-33000][sql-gateway] SqlGatewayServiceITCase should utilize TestExecutorExtension instead of using a ThreadFactory new 73717520cc6 [FLINK-33000][sql-gateway] OperationManagerTest should utilize TestExecutorExtension instead of using a ThreadFactory new 8e68b40d5cf [FLINK-33000][sql-gateway] ResultFetcherTest should utilize TestExecutorExtension instead of using a ThreadFactory The 3 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference. Summary of changes: .../gateway/service/SqlGatewayServiceITCase.java | 59 +- .../service/operation/OperationManagerTest.java| 43 .../gateway/service/result/ResultFetcherTest.java | 51 +-- 3 files changed, 91 insertions(+), 62 deletions(-)
[flink] 03/03: [FLINK-33000][sql-gateway] ResultFetcherTest should utilize TestExecutorExtension instead of using a ThreadFactory
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 8e68b40d5cf9146a5bccadc67d9636f987003c0f Author: Jiabao Sun AuthorDate: Thu Sep 21 17:55:51 2023 +0800 [FLINK-33000][sql-gateway] ResultFetcherTest should utilize TestExecutorExtension instead of using a ThreadFactory --- .../gateway/service/result/ResultFetcherTest.java | 51 +++--- 1 file changed, 35 insertions(+), 16 deletions(-) diff --git a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/result/ResultFetcherTest.java b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/result/ResultFetcherTest.java index c5b8bf3c6a3..e95d9fdb5a1 100644 --- a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/result/ResultFetcherTest.java +++ b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/result/ResultFetcherTest.java @@ -31,6 +31,7 @@ import org.apache.flink.table.gateway.api.results.FetchOrientation; import org.apache.flink.table.gateway.api.results.ResultSet; import org.apache.flink.table.gateway.service.utils.IgnoreExceptionHandler; import org.apache.flink.table.gateway.service.utils.SqlExecutionException; +import org.apache.flink.testutils.executor.TestExecutorExtension; import org.apache.flink.types.RowKind; import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.concurrent.ExecutorThreadFactory; @@ -38,6 +39,7 @@ import org.apache.flink.util.concurrent.ExecutorThreadFactory; import org.apache.commons.collections.iterators.IteratorChain; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; import java.math.BigDecimal; import java.sql.Timestamp; @@ -51,7 +53,8 @@ import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ThreadFactory; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.function.Supplier; @@ -68,8 +71,14 @@ class ResultFetcherTest { private static ResolvedSchema schema; private static List data; -private final ThreadFactory threadFactory = -new ExecutorThreadFactory("Result Fetcher Test Pool", IgnoreExceptionHandler.INSTANCE); +@RegisterExtension +private static final TestExecutorExtension EXECUTOR_EXTENSION = +new TestExecutorExtension<>( +() -> +Executors.newCachedThreadPool( +new ExecutorThreadFactory( +"Result Fetcher Test Pool", +IgnoreExceptionHandler.INSTANCE))); @BeforeAll static void setUp() { @@ -243,8 +252,9 @@ class ResultFetcherTest { AtomicReference payloadHasData = new AtomicReference<>(true); for (int i = 0; i < fetchThreadNum; i++) { -threadFactory -.newThread( +EXECUTOR_EXTENSION +.getExecutor() +.submit( () -> { ResultSet resultSet = fetcher.fetchResults(FetchOrientation.FETCH_NEXT, 1); @@ -253,10 +263,19 @@ class ResultFetcherTest { payloadHasData.set(false); } -rows.put(Thread.currentThread().getId(), resultSet.getData()); +rows.compute( +Thread.currentThread().getId(), +(k, v) -> { +if (v == null) { +return resultSet.getData(); +} else { +v.addAll(resultSet.getData()); +return v; +} +}); + latch.countDown(); -}) -.start(); +}); } latch.await(); @@ -290,8 +309,9 @@ class ResultFetcherTest { long testToken = token; AtomicReference meetEnd = new AtomicReference<>(false); -threadFactory -.newThread( +EXECUTOR_EXTENSION +
[flink] branch master updated: Revert "[FLINK-33064][table-planner] Improve the error message when the lookup source is used as the scan source (#23377)" (#23463)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new 51e5ce6f4b9 Revert "[FLINK-33064][table-planner] Improve the error message when the lookup source is used as the scan source (#23377)" (#23463) 51e5ce6f4b9 is described below commit 51e5ce6f4b9c889e7e253b3b8eba3ca8ee6af4e8 Author: yunhong <337361...@qq.com> AuthorDate: Tue Sep 26 17:09:55 2023 +0800 Revert "[FLINK-33064][table-planner] Improve the error message when the lookup source is used as the scan source (#23377)" (#23463) This reverts commit be509e6d67471d886e58d3ddea6ddd3627a191a8. Co-authored-by: zhengyunhong.zyh --- .../TestDynamicTableSourceOnlyFactory.java | 41 + .../apache/calcite/sql2rel/SqlToRelConverter.java | 172 - .../planner/plan/utils/TemporalTableJoinUtil.java | 14 -- .../rules/common/CommonTemporalTableJoinRule.scala | 17 +- .../table/planner/catalog/CatalogTableITCase.scala | 52 +++ .../planner/plan/stream/sql/TableScanTest.scala| 8 +- 6 files changed, 68 insertions(+), 236 deletions(-) diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/factories/TestDynamicTableSourceOnlyFactory.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/factories/TestDynamicTableSourceOnlyFactory.java index 6d3af4650d3..7aa78b6492e 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/factories/TestDynamicTableSourceOnlyFactory.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/factories/TestDynamicTableSourceOnlyFactory.java @@ -19,10 +19,7 @@ package org.apache.flink.table.factories; import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.connector.source.ScanTableSource; import java.util.Collections; import java.util.Set; @@ -35,14 +32,9 @@ public final class TestDynamicTableSourceOnlyFactory implements DynamicTableSour public static final String IDENTIFIER = "source-only"; -private static final ConfigOption BOUNDED = -ConfigOptions.key("bounded").booleanType().defaultValue(false); - @Override public DynamicTableSource createDynamicTableSource(Context context) { -FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); -boolean isBounded = helper.getOptions().get(BOUNDED); -return new MockedScanTableSource(isBounded); +return null; } @Override @@ -57,35 +49,6 @@ public final class TestDynamicTableSourceOnlyFactory implements DynamicTableSour @Override public Set> optionalOptions() { -return Collections.singleton(BOUNDED); -} - -/** A mocked {@link ScanTableSource} for validation test. */ -private static class MockedScanTableSource implements ScanTableSource { -private final boolean isBounded; - -private MockedScanTableSource(boolean isBounded) { -this.isBounded = isBounded; -} - -@Override -public DynamicTableSource copy() { -return null; -} - -@Override -public String asSummaryString() { -return null; -} - -@Override -public ChangelogMode getChangelogMode() { -return ChangelogMode.insertOnly(); -} - -@Override -public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) { -return () -> isBounded; -} +return Collections.emptySet(); } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java index 67663ce0984..71cafa91308 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java @@ -17,16 +17,11 @@ package org.apache.calcite.sql2rel; import org.apache.flink.table.api.TableConfig; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.connector.source.ScanTableSource; import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.planner.alias.ClearJoinHintWithInvalidPropagationShuttle; import org.apache.flink.table.planner.calcite.TimestampSchemaVersion; import org
[flink] branch master updated (c5241e502ef -> cbfb9fa3ba9)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from c5241e502ef [FLINK-33061][docs] Translate failure-enricher documentation to Chinese add cbfb9fa3ba9 [hotfix][test] Fix test value source throw ClassCastException while push down none-existent partition (#23414) No new revisions were added by this update. Summary of changes: .../planner/factories/TestValuesTableFactory.java | 8 ++- .../plan/batch/sql/PartitionableSourceTest.xml | 71 +++--- .../plan/stream/sql/PartitionableSourceTest.xml| 71 +++--- .../plan/batch/sql/PartitionableSourceTest.scala | 5 ++ .../plan/stream/sql/PartitionableSourceTest.scala | 6 ++ 5 files changed, 144 insertions(+), 17 deletions(-)
[flink] branch master updated: [FLINK-33064][table-planner] Improve the error message when the lookup source is used as the scan source (#23377)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new be509e6d674 [FLINK-33064][table-planner] Improve the error message when the lookup source is used as the scan source (#23377) be509e6d674 is described below commit be509e6d67471d886e58d3ddea6ddd3627a191a8 Author: yunhong <337361...@qq.com> AuthorDate: Wed Sep 13 13:58:04 2023 +0800 [FLINK-33064][table-planner] Improve the error message when the lookup source is used as the scan source (#23377) Co-authored-by: zhengyunhong.zyh --- .../TestDynamicTableSourceOnlyFactory.java | 41 - .../apache/calcite/sql2rel/SqlToRelConverter.java | 172 + .../planner/plan/utils/TemporalTableJoinUtil.java | 14 ++ .../rules/common/CommonTemporalTableJoinRule.scala | 17 +- .../table/planner/catalog/CatalogTableITCase.scala | 52 --- .../planner/plan/stream/sql/TableScanTest.scala| 8 +- 6 files changed, 236 insertions(+), 68 deletions(-) diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/factories/TestDynamicTableSourceOnlyFactory.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/factories/TestDynamicTableSourceOnlyFactory.java index 7aa78b6492e..6d3af4650d3 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/factories/TestDynamicTableSourceOnlyFactory.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/factories/TestDynamicTableSourceOnlyFactory.java @@ -19,7 +19,10 @@ package org.apache.flink.table.factories; import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; import java.util.Collections; import java.util.Set; @@ -32,9 +35,14 @@ public final class TestDynamicTableSourceOnlyFactory implements DynamicTableSour public static final String IDENTIFIER = "source-only"; +private static final ConfigOption BOUNDED = +ConfigOptions.key("bounded").booleanType().defaultValue(false); + @Override public DynamicTableSource createDynamicTableSource(Context context) { -return null; +FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); +boolean isBounded = helper.getOptions().get(BOUNDED); +return new MockedScanTableSource(isBounded); } @Override @@ -49,6 +57,35 @@ public final class TestDynamicTableSourceOnlyFactory implements DynamicTableSour @Override public Set> optionalOptions() { -return Collections.emptySet(); +return Collections.singleton(BOUNDED); +} + +/** A mocked {@link ScanTableSource} for validation test. */ +private static class MockedScanTableSource implements ScanTableSource { +private final boolean isBounded; + +private MockedScanTableSource(boolean isBounded) { +this.isBounded = isBounded; +} + +@Override +public DynamicTableSource copy() { +return null; +} + +@Override +public String asSummaryString() { +return null; +} + +@Override +public ChangelogMode getChangelogMode() { +return ChangelogMode.insertOnly(); +} + +@Override +public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) { +return () -> isBounded; +} } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java index 71cafa91308..67663ce0984 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java @@ -17,11 +17,16 @@ package org.apache.calcite.sql2rel; import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.planner.alias.ClearJoinHintWithInvalidPropagationShuttle; import org.apache.flink.table.planner.calcite.TimestampSchemaVersion; import org.apache.flink.table.planner.hint.FlinkHints; import org.apache.flink.table.planner.plan.FlinkCalciteCatalogSnapshotReader; +import org.apache.flink.table.plan
[flink] branch release-1.18 updated (73632fa7ba7 -> 9e5659ea652)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch release-1.18 in repository https://gitbox.apache.org/repos/asf/flink.git from 73632fa7ba7 [FLINK-30025][doc] improve the option description with more precise content new 3dcdc7f2938 [FLINK-32731][e2e] Add retry mechanism when fails to start the namenode (#23267) new 9e5659ea652 [FLINK-32731][e2e] Fix NameNode uses port 9870 in hadoop3 The 2 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference. Summary of changes: .../org/apache/flink/tests/hive/HiveITCase.java| 6 +- .../flink/tests/hive/containers/HiveContainer.java | 158 - .../tests/hive/containers/HiveContainers.java | 246 + .../table/gateway/containers/HiveContainer.java| 30 +++ 4 files changed, 279 insertions(+), 161 deletions(-) delete mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/containers/HiveContainer.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/containers/HiveContainers.java
[flink] 01/02: [FLINK-32731][e2e] Add retry mechanism when fails to start the namenode (#23267)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch release-1.18 in repository https://gitbox.apache.org/repos/asf/flink.git commit 3dcdc7f29384bc399e65ce46253975570e93481f Author: Shengkai <33114724+fsk...@users.noreply.github.com> AuthorDate: Mon Aug 28 09:38:20 2023 +0800 [FLINK-32731][e2e] Add retry mechanism when fails to start the namenode (#23267) --- .../flink/tests/hive/containers/HiveContainer.java | 30 ++ .../table/gateway/containers/HiveContainer.java| 30 ++ 2 files changed, 60 insertions(+) diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/containers/HiveContainer.java b/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/containers/HiveContainer.java index 6edb7f46e8c..d4ca4080023 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/containers/HiveContainer.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/containers/HiveContainer.java @@ -22,6 +22,11 @@ import org.apache.flink.table.catalog.hive.client.HiveShimLoader; import org.apache.flink.test.parameters.ParameterProperty; import org.apache.flink.util.DockerImageVersions; +import com.github.dockerjava.api.command.InspectContainerResponse; +import okhttp3.FormBody; +import okhttp3.OkHttpClient; +import okhttp3.Request; +import okhttp3.Response; import org.junit.runner.Description; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,6 +61,7 @@ public class HiveContainer extends GenericContainer { private static final String MYSQL_METASTORE_LOG_PATH = "/var/log/mysqld.log"; private static final ParameterProperty DISTRIBUTION_LOG_BACKUP_DIRECTORY = new ParameterProperty<>("logBackupDir", Paths::get); +private static final int NAME_NODE_WEB_PORT = 50070; private String hiveWarehouseDir; @@ -64,7 +70,9 @@ public class HiveContainer extends GenericContainer { DockerImageName.parse( HIVE_310_OR_LATER ? DockerImageVersions.HIVE3 : DockerImageVersions.HIVE2)); withExtraHost(HOST_NAME, "127.0.0.1"); +withStartupAttempts(3); addExposedPort(HIVE_METASTORE_PORT); +addExposedPort(NAME_NODE_WEB_PORT); mountHiveWarehouseDirToContainer(initTableNames); } @@ -82,6 +90,28 @@ public class HiveContainer extends GenericContainer { super.finished(description); } +@Override +protected void containerIsStarted(InspectContainerResponse containerInfo) { +super.containerIsStarted(containerInfo); +final Request request = +new Request.Builder() +.post(new FormBody.Builder().build()) +.url( +String.format( +"http://127.0.0.1:%s";, getMappedPort(NAME_NODE_WEB_PORT))) +.build(); +OkHttpClient client = new OkHttpClient(); +try (Response response = client.newCall(request).execute()) { +if (!response.isSuccessful()) { +throw new RuntimeException( +String.format( +"The rest request is not successful: %s", response.message())); +} +} catch (Exception e) { +throw new RuntimeException(e); +} +} + public String getHiveMetastoreURL() { return String.format("thrift://%s:%s", getHost(), getMappedPort(HIVE_METASTORE_PORT)); } diff --git a/flink-end-to-end-tests/flink-sql-gateway-test/src/test/java/org/apache/flink/table/gateway/containers/HiveContainer.java b/flink-end-to-end-tests/flink-sql-gateway-test/src/test/java/org/apache/flink/table/gateway/containers/HiveContainer.java index c16aa41b8db..cb44cd4e517 100644 --- a/flink-end-to-end-tests/flink-sql-gateway-test/src/test/java/org/apache/flink/table/gateway/containers/HiveContainer.java +++ b/flink-end-to-end-tests/flink-sql-gateway-test/src/test/java/org/apache/flink/table/gateway/containers/HiveContainer.java @@ -20,6 +20,11 @@ package org.apache.flink.table.gateway.containers; import org.apache.flink.test.parameters.ParameterProperty; import org.apache.flink.util.DockerImageVersions; +import com.github.dockerjava.api.command.InspectContainerResponse; +import okhttp3.FormBody; +import okhttp3.OkHttpClient; +import okhttp3.Request; +import okhttp3.Response; import org.junit.runner.Description; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,6 +44,7 @@ public class HiveContainer extends GenericContainer { public static final String HOST_NAME = "hadoop-master"; public static final int HI
[flink] 02/02: [FLINK-32731][e2e] Fix NameNode uses port 9870 in hadoop3
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch release-1.18 in repository https://gitbox.apache.org/repos/asf/flink.git commit 9e5659ea65278b2b699ab0c0f0eafc918a0107bc Author: Shengkai <1059623...@qq.com> AuthorDate: Thu Sep 7 17:55:31 2023 +0800 [FLINK-32731][e2e] Fix NameNode uses port 9870 in hadoop3 This closes #23370 --- .../org/apache/flink/tests/hive/HiveITCase.java| 6 +- .../flink/tests/hive/containers/HiveContainer.java | 188 .../tests/hive/containers/HiveContainers.java | 246 + 3 files changed, 249 insertions(+), 191 deletions(-) diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/HiveITCase.java b/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/HiveITCase.java index 5b310688f89..24a759887c5 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/HiveITCase.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/HiveITCase.java @@ -22,7 +22,7 @@ import org.apache.flink.api.common.time.Deadline; import org.apache.flink.table.catalog.hive.HiveCatalog; import org.apache.flink.test.resources.ResourceTestUtils; import org.apache.flink.test.util.SQLJobSubmission; -import org.apache.flink.tests.hive.containers.HiveContainer; +import org.apache.flink.tests.hive.containers.HiveContainers; import org.apache.flink.tests.util.flink.ClusterController; import org.apache.flink.tests.util.flink.FlinkResource; import org.apache.flink.tests.util.flink.FlinkResourceSetup; @@ -69,8 +69,8 @@ public class HiveITCase extends TestLogger { @ClassRule public static final TemporaryFolder TMP_FOLDER = new TemporaryFolder(); @ClassRule -public static final HiveContainer HIVE_CONTAINER = -new HiveContainer( +public static final HiveContainers.HiveContainer HIVE_CONTAINER = +HiveContainers.createHiveContainer( Arrays.asList("hive_sink1", "hive_sink2", "h_table_sink1", "h_table_sink2")); private static final String HIVE_ADD_ONE_UDF_CLASS = "HiveAddOneFunc"; diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/containers/HiveContainer.java b/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/containers/HiveContainer.java deleted file mode 100644 index d4ca4080023..000 --- a/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/containers/HiveContainer.java +++ /dev/null @@ -1,188 +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.hive.containers; - -import org.apache.flink.table.catalog.hive.client.HiveShimLoader; -import org.apache.flink.test.parameters.ParameterProperty; -import org.apache.flink.util.DockerImageVersions; - -import com.github.dockerjava.api.command.InspectContainerResponse; -import okhttp3.FormBody; -import okhttp3.OkHttpClient; -import okhttp3.Request; -import okhttp3.Response; -import org.junit.runner.Description; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testcontainers.containers.GenericContainer; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.utility.DockerImageName; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.List; -import java.util.UUID; - -/** Test container for Hive. */ -public class HiveContainer extends GenericContainer { - -private static final Logger LOG = LoggerFactory.getLogger(HiveContainer.class); -public static final String HOST_NAME = "hadoop-master"; -public static final int HIVE_METASTORE_PORT = 9083; - -private static final boolean HIVE_310_OR_LATER = - HiveShimLoader.getHiveVersion().compareTo(HiveShimLoader.HIVE_VERSION_V3_1_0) >= 0; - -// D
[flink] branch release-1.18 updated: [FLINK-33063][table-runtime] Fix udaf with complex user defined pojo object throw error while generate record equaliser (#23388)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch release-1.18 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.18 by this push: new f2584a1df36 [FLINK-33063][table-runtime] Fix udaf with complex user defined pojo object throw error while generate record equaliser (#23388) f2584a1df36 is described below commit f2584a1df364a14ff50b5a52fe7cf5e38d4cdc9a Author: yunhong <337361...@qq.com> AuthorDate: Tue Sep 12 11:08:48 2023 +0800 [FLINK-33063][table-runtime] Fix udaf with complex user defined pojo object throw error while generate record equaliser (#23388) Co-authored-by: zhengyunhong.zyh --- .../plan/utils/JavaUserDefinedAggFunctions.java| 80 ++ .../runtime/stream/sql/AggregateITCase.scala | 30 +++- .../table/runtime/typeutils/TypeCheckUtils.java| 12 +++- 3 files changed, 120 insertions(+), 2 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/JavaUserDefinedAggFunctions.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/JavaUserDefinedAggFunctions.java index 64cb9bbd2bd..4e722be8ee3 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/JavaUserDefinedAggFunctions.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/JavaUserDefinedAggFunctions.java @@ -25,7 +25,11 @@ import org.apache.flink.table.api.dataview.ListView; import org.apache.flink.table.api.dataview.MapView; import org.apache.flink.table.functions.AggregateFunction; +import java.util.ArrayList; +import java.util.HashMap; import java.util.Iterator; +import java.util.List; +import java.util.Map; /** Test aggregator functions. */ public class JavaUserDefinedAggFunctions { @@ -421,4 +425,80 @@ public class JavaUserDefinedAggFunctions { return Tuple1.of(0L); } } + +/** User defined pojo object. */ +public static class TestObject { +private final String a; + +public TestObject(String a) { +this.a = a; +} + +public String getA() { +return a; +} +} + +/** User defined object. */ +public static class UserDefinedObject { +// List with user defined pojo object. +public List testObjectList = new ArrayList<>(); +// Map with user defined pojo object. +public Map testObjectMap = new HashMap<>(); +} + +/** User defined UDAF whose value and acc is user defined complex pojo object. */ +public static class UserDefinedObjectUDAF +extends AggregateFunction { +private static final String KEY = "key"; + +@Override +public UserDefinedObject getValue(UserDefinedObject accumulator) { +return accumulator; +} + +@Override +public UserDefinedObject createAccumulator() { +return new UserDefinedObject(); +} + +public void accumulate(UserDefinedObject acc, String a) { +if (a != null) { +acc.testObjectList.add(new TestObject(a)); +acc.testObjectMap.put(KEY, new TestObject(a)); +} +} + +public void retract(UserDefinedObject acc, UserDefinedObject a) { +// do nothing. +} +} + +/** User defined UDAF whose value and acc is user defined complex pojo object. */ +public static class UserDefinedObjectUDAF2 +extends AggregateFunction { +private static final String KEY = "key"; + +@Override +public String getValue(UserDefinedObject accumulator) { +if (accumulator.testObjectMap.containsKey(KEY)) { +return accumulator.testObjectMap.get(KEY).getA(); +} +return null; +} + +@Override +public UserDefinedObject createAccumulator() { +return new UserDefinedObject(); +} + +public void accumulate(UserDefinedObject acc, UserDefinedObject a) { +acc.testObjectList = a.testObjectList; +acc.testObjectMap = a.testObjectMap; +} + +public void retract(UserDefinedObject acc, UserDefinedObject a) { +// do nothing +} +} } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateITCase.scala index 152c8d02be4..4df20e1615d 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/
[flink] branch release-1.18 updated: [FLINK-32952][table-planner] Fix scan reuse with readable metadata and watermark push down get wrong watermark error (#23338)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch release-1.18 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.18 by this push: new f66679bfe5f [FLINK-32952][table-planner] Fix scan reuse with readable metadata and watermark push down get wrong watermark error (#23338) f66679bfe5f is described below commit f66679bfe5f5b344eec71a7579504762cc3c04ae Author: yunhong <337361...@qq.com> AuthorDate: Tue Sep 12 11:07:23 2023 +0800 [FLINK-32952][table-planner] Fix scan reuse with readable metadata and watermark push down get wrong watermark error (#23338) Co-authored-by: zhengyunhong.zyh --- .../flink/table/planner/plan/reuse/ScanReuser.java | 16 +- .../table/planner/plan/reuse/ScanReuserUtils.java | 12 +- .../table/planner/plan/optimize/ScanReuseTest.java | 76 .../table/planner/plan/optimize/ScanReuseTest.xml | 214 +++-- 4 files changed, 243 insertions(+), 75 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/reuse/ScanReuser.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/reuse/ScanReuser.java index 1ef3bf2f2b1..9128e110346 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/reuse/ScanReuser.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/reuse/ScanReuser.java @@ -170,12 +170,18 @@ public class ScanReuser { // 2. Create new source. List specs = abilitySpecsWithoutEscaped(pickTable); -// 2.1 Apply projections -List newSpecs = new ArrayList<>(); +// 2.1 Create produced type. +// The source produced type is the input type into the runtime. The format looks as: +// PHYSICAL COLUMNS + METADATA COLUMNS. While re-compute the source ability specs with +// source metadata, we need to distinguish between schema type and produced type, which +// source ability specs use produced type instead of schema type. RowType originType = DynamicSourceUtils.createProducedType( pickTable.contextResolvedTable().getResolvedSchema(), pickTable.tableSource()); + +// 2.2 Apply projections +List newSpecs = new ArrayList<>(); RowType newSourceType = applyPhysicalAndMetadataPushDown( pickTable.tableSource(), @@ -190,15 +196,15 @@ public class ScanReuser { allMetaKeys); specs.addAll(newSpecs); -// 2.2 Watermark spec +// 2.3 Watermark spec Optional watermarkSpec = -getAdjustedWatermarkSpec(pickTable, newSourceType); +getAdjustedWatermarkSpec(pickTable, originType, newSourceType); if (watermarkSpec.isPresent()) { specs.add(watermarkSpec.get()); newSourceType = watermarkSpec.get().getProducedType().get(); } -// 2.3 Create a new ScanTableSource. ScanTableSource can not be pushed down twice. +// 2.4 Create a new ScanTableSource. ScanTableSource can not be pushed down twice. DynamicTableSourceSpec tableSourceSpec = new DynamicTableSourceSpec(pickTable.contextResolvedTable(), specs); ScanTableSource newTableSource = diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/reuse/ScanReuserUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/reuse/ScanReuserUtils.java index 5c79cd4cd25..43a00d720a2 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/reuse/ScanReuserUtils.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/reuse/ScanReuserUtils.java @@ -170,24 +170,18 @@ public class ScanReuserUtils { /** Watermark push down must be after projection push down, so we need to adjust its index. */ public static Optional getAdjustedWatermarkSpec( -TableSourceTable table, RowType newSourceType) { -RowType producedType = -(RowType) -table.contextResolvedTable() -.getResolvedSchema() -.toSourceRowDataType() -.getLogicalType(); +TableSourceTable table, RowType oldSourceType, RowType newSourceType) { for (SourceAbilitySpec spec : table.abilitySpecs()) { if (spec instanceof WatermarkPushDownSpec) { return Optional.of( ad
[flink] branch master updated: [FLINK-32731][e2e] Fix NameNode uses port 9870 in hadoop3
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new 649b7fe197c [FLINK-32731][e2e] Fix NameNode uses port 9870 in hadoop3 649b7fe197c is described below commit 649b7fe197c8b03cce9595adcfea33c8d708a8b4 Author: Shengkai <1059623...@qq.com> AuthorDate: Thu Sep 7 17:55:31 2023 +0800 [FLINK-32731][e2e] Fix NameNode uses port 9870 in hadoop3 This closes #23370 --- .../org/apache/flink/tests/hive/HiveITCase.java| 6 +- .../flink/tests/hive/containers/HiveContainer.java | 188 .../tests/hive/containers/HiveContainers.java | 246 + 3 files changed, 249 insertions(+), 191 deletions(-) diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/HiveITCase.java b/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/HiveITCase.java index 5b310688f89..24a759887c5 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/HiveITCase.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/HiveITCase.java @@ -22,7 +22,7 @@ import org.apache.flink.api.common.time.Deadline; import org.apache.flink.table.catalog.hive.HiveCatalog; import org.apache.flink.test.resources.ResourceTestUtils; import org.apache.flink.test.util.SQLJobSubmission; -import org.apache.flink.tests.hive.containers.HiveContainer; +import org.apache.flink.tests.hive.containers.HiveContainers; import org.apache.flink.tests.util.flink.ClusterController; import org.apache.flink.tests.util.flink.FlinkResource; import org.apache.flink.tests.util.flink.FlinkResourceSetup; @@ -69,8 +69,8 @@ public class HiveITCase extends TestLogger { @ClassRule public static final TemporaryFolder TMP_FOLDER = new TemporaryFolder(); @ClassRule -public static final HiveContainer HIVE_CONTAINER = -new HiveContainer( +public static final HiveContainers.HiveContainer HIVE_CONTAINER = +HiveContainers.createHiveContainer( Arrays.asList("hive_sink1", "hive_sink2", "h_table_sink1", "h_table_sink2")); private static final String HIVE_ADD_ONE_UDF_CLASS = "HiveAddOneFunc"; diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/containers/HiveContainer.java b/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/containers/HiveContainer.java deleted file mode 100644 index d4ca4080023..000 --- a/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/containers/HiveContainer.java +++ /dev/null @@ -1,188 +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.hive.containers; - -import org.apache.flink.table.catalog.hive.client.HiveShimLoader; -import org.apache.flink.test.parameters.ParameterProperty; -import org.apache.flink.util.DockerImageVersions; - -import com.github.dockerjava.api.command.InspectContainerResponse; -import okhttp3.FormBody; -import okhttp3.OkHttpClient; -import okhttp3.Request; -import okhttp3.Response; -import org.junit.runner.Description; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testcontainers.containers.GenericContainer; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.utility.DockerImageName; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.List; -import java.util.UUID; - -/** Test container for Hive. */ -public class HiveContainer extends GenericContainer { - -private static final Logger LOG = LoggerFactory.getLogger(HiveContainer.class); -public static final String HOST_NAME = "hadoop-master"; -public static final
[flink] branch master updated: [FLINK-32731][e2e] Add retry mechanism when fails to start the namenode (#23267)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new 4b84b6cd598 [FLINK-32731][e2e] Add retry mechanism when fails to start the namenode (#23267) 4b84b6cd598 is described below commit 4b84b6cd5983ae8f058fae731eb0f4af6214b738 Author: Shengkai <33114724+fsk...@users.noreply.github.com> AuthorDate: Mon Aug 28 09:38:20 2023 +0800 [FLINK-32731][e2e] Add retry mechanism when fails to start the namenode (#23267) --- .../flink/tests/hive/containers/HiveContainer.java | 30 ++ .../table/gateway/containers/HiveContainer.java| 30 ++ 2 files changed, 60 insertions(+) diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/containers/HiveContainer.java b/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/containers/HiveContainer.java index 6edb7f46e8c..d4ca4080023 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/containers/HiveContainer.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/containers/HiveContainer.java @@ -22,6 +22,11 @@ import org.apache.flink.table.catalog.hive.client.HiveShimLoader; import org.apache.flink.test.parameters.ParameterProperty; import org.apache.flink.util.DockerImageVersions; +import com.github.dockerjava.api.command.InspectContainerResponse; +import okhttp3.FormBody; +import okhttp3.OkHttpClient; +import okhttp3.Request; +import okhttp3.Response; import org.junit.runner.Description; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,6 +61,7 @@ public class HiveContainer extends GenericContainer { private static final String MYSQL_METASTORE_LOG_PATH = "/var/log/mysqld.log"; private static final ParameterProperty DISTRIBUTION_LOG_BACKUP_DIRECTORY = new ParameterProperty<>("logBackupDir", Paths::get); +private static final int NAME_NODE_WEB_PORT = 50070; private String hiveWarehouseDir; @@ -64,7 +70,9 @@ public class HiveContainer extends GenericContainer { DockerImageName.parse( HIVE_310_OR_LATER ? DockerImageVersions.HIVE3 : DockerImageVersions.HIVE2)); withExtraHost(HOST_NAME, "127.0.0.1"); +withStartupAttempts(3); addExposedPort(HIVE_METASTORE_PORT); +addExposedPort(NAME_NODE_WEB_PORT); mountHiveWarehouseDirToContainer(initTableNames); } @@ -82,6 +90,28 @@ public class HiveContainer extends GenericContainer { super.finished(description); } +@Override +protected void containerIsStarted(InspectContainerResponse containerInfo) { +super.containerIsStarted(containerInfo); +final Request request = +new Request.Builder() +.post(new FormBody.Builder().build()) +.url( +String.format( +"http://127.0.0.1:%s";, getMappedPort(NAME_NODE_WEB_PORT))) +.build(); +OkHttpClient client = new OkHttpClient(); +try (Response response = client.newCall(request).execute()) { +if (!response.isSuccessful()) { +throw new RuntimeException( +String.format( +"The rest request is not successful: %s", response.message())); +} +} catch (Exception e) { +throw new RuntimeException(e); +} +} + public String getHiveMetastoreURL() { return String.format("thrift://%s:%s", getHost(), getMappedPort(HIVE_METASTORE_PORT)); } diff --git a/flink-end-to-end-tests/flink-sql-gateway-test/src/test/java/org/apache/flink/table/gateway/containers/HiveContainer.java b/flink-end-to-end-tests/flink-sql-gateway-test/src/test/java/org/apache/flink/table/gateway/containers/HiveContainer.java index c16aa41b8db..cb44cd4e517 100644 --- a/flink-end-to-end-tests/flink-sql-gateway-test/src/test/java/org/apache/flink/table/gateway/containers/HiveContainer.java +++ b/flink-end-to-end-tests/flink-sql-gateway-test/src/test/java/org/apache/flink/table/gateway/containers/HiveContainer.java @@ -20,6 +20,11 @@ package org.apache.flink.table.gateway.containers; import org.apache.flink.test.parameters.ParameterProperty; import org.apache.flink.util.DockerImageVersions; +import com.github.dockerjava.api.command.InspectContainerResponse; +import okhttp3.FormBody; +import okhttp3.OkHttpClient; +import okhttp3.Request; +import okhttp3.Response; import org.junit.runner.Description; import org.slf4j.Logger; import org.s
[flink] branch master updated (2eb88b4115f -> 9231d62d25d)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 2eb88b4115f [FLINK-32287][docs] Add doc for truncate table statement (#23046) add 9231d62d25d [FLINK-25447][table-planner] Fix batch query cannot generate plan when inserting into multi sinks from a sorted view (#20822) No new revisions were added by this update. Summary of changes: .../plan/metadata/FlinkRelMdDistribution.scala | 14 ++- .../planner/plan/schema/IntermediateRelTable.scala | 17 ++-- .../planner/plan/batch/sql/SubplanReuseTest.xml| 37 + .../planner/plan/batch/sql/SubplanReuseTest.scala | 46 ++ .../plan/metadata/FlinkRelMdDistributionTest.scala | 8 .../plan/metadata/FlinkRelMdHandlerTestBase.scala | 37 + .../plan/metadata/FlinkRelMdRowCollationTest.scala | 21 ++ 7 files changed, 176 insertions(+), 4 deletions(-)
[flink] branch master updated: [FLINK-32731][e2e] Backup the hive-metastore, hdfs namenode and mysql logs (#23155)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new 5654eb798c7 [FLINK-32731][e2e] Backup the hive-metastore, hdfs namenode and mysql logs (#23155) 5654eb798c7 is described below commit 5654eb798c744c924aff93d68ec3c4e413e75232 Author: Shengkai <33114724+fsk...@users.noreply.github.com> AuthorDate: Wed Aug 9 15:46:07 2023 +0800 [FLINK-32731][e2e] Backup the hive-metastore, hdfs namenode and mysql logs (#23155) --- .../flink/tests/hive/containers/HiveContainer.java | 58 +- .../table/gateway/containers/HiveContainer.java| 57 + 2 files changed, 113 insertions(+), 2 deletions(-) diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/containers/HiveContainer.java b/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/containers/HiveContainer.java index 716f40d48e1..6edb7f46e8c 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/containers/HiveContainer.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-hive/src/test/java/org/apache/flink/tests/hive/containers/HiveContainer.java @@ -19,8 +19,10 @@ package org.apache.flink.tests.hive.containers; import org.apache.flink.table.catalog.hive.client.HiveShimLoader; +import org.apache.flink.test.parameters.ParameterProperty; import org.apache.flink.util.DockerImageVersions; +import org.junit.runner.Description; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testcontainers.containers.GenericContainer; @@ -31,19 +33,31 @@ import java.io.File; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; +import java.nio.file.Paths; import java.util.List; +import java.util.UUID; /** Test container for Hive. */ public class HiveContainer extends GenericContainer { private static final Logger LOG = LoggerFactory.getLogger(HiveContainer.class); public static final String HOST_NAME = "hadoop-master"; +public static final int HIVE_METASTORE_PORT = 9083; private static final boolean HIVE_310_OR_LATER = HiveShimLoader.getHiveVersion().compareTo(HiveShimLoader.HIVE_VERSION_V3_1_0) >= 0; -private String hiveWarehouseDir; -public static final int HIVE_METASTORE_PORT = 9083; +// Detailed log paths are from +// https://github.com/prestodb/docker-images/tree/master/prestodb/hdp2.6-hive/files/etc/supervisord.d +// https://github.com/prestodb/docker-images/blob/master/prestodb/hive3.1-hive/files/etc/supervisord.conf +private static final String NAME_NODE_LOG_PATH = +"/var/log/hadoop-hdfs/hadoop-hdfs-namenode.log"; +private static final String METASTORE_LOG_PATH = "/tmp/hive/hive.log"; +private static final String MYSQL_METASTORE_LOG_PATH = "/var/log/mysqld.log"; +private static final ParameterProperty DISTRIBUTION_LOG_BACKUP_DIRECTORY = +new ParameterProperty<>("logBackupDir", Paths::get); + +private String hiveWarehouseDir; public HiveContainer(List initTableNames) { super( @@ -62,6 +76,12 @@ public class HiveContainer extends GenericContainer { } } +@Override +protected void finished(Description description) { +backupLogs(); +super.finished(description); +} + public String getHiveMetastoreURL() { return String.format("thrift://%s:%s", getHost(), getMappedPort(HIVE_METASTORE_PORT)); } @@ -101,4 +121,38 @@ public class HiveContainer extends GenericContainer { file.setWritable(true, false); file.setExecutable(true, false); } + +private void backupLogs() { +Path path = DISTRIBUTION_LOG_BACKUP_DIRECTORY.get().orElse(null); +if (path == null) { +LOG.warn( +"Property {} not set, logs will not be backed up.", +DISTRIBUTION_LOG_BACKUP_DIRECTORY.getPropertyName()); +return; +} +try { +Path dir = +Files.createDirectory( +Paths.get( +String.valueOf(path.toAbsolutePath()), +"hive-" + UUID.randomUUID())); +copyFileFromContainer( +NAME_NODE_LOG_PATH, + Files.createFile(Paths.get(dir.toAbsolutePath().toString(), "namenode.log")) +.toAbsolutePath() +.toString()); +copyFileFromContainer( +METASTORE_LOG_PATH, + Files.createFile(
[flink] branch master updated (a5a741876ab -> b9c5dc3d731)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from a5a741876ab [FLINK-32617][jdbc-driver] Return true or false for most methods in database meta data (#23019) add b9c5dc3d731 [FLINK-32616][jdbc-driver] Close result for non-query in executeQuery (#23027) No new revisions were added by this update. Summary of changes: .../apache/flink/table/jdbc/FlinkConnection.java | 7 ++ .../apache/flink/table/jdbc/FlinkStatement.java| 1 + .../flink/table/jdbc/FlinkStatementTest.java | 80 ++ 3 files changed, 88 insertions(+)
[flink] branch master updated (05191071638 -> a5a741876ab)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 05191071638 [FLINK-32456][table-planner] Support JSON_OBJECTAGG & JSON_ARRAYAGG use with other aggregate functions add a5a741876ab [FLINK-32617][jdbc-driver] Return true or false for most methods in database meta data (#23019) No new revisions were added by this update. Summary of changes: .../flink/table/jdbc/BaseDatabaseMetaData.java | 408 - .../flink/table/jdbc/FlinkDatabaseMetaData.java| 347 ++ .../table/jdbc/FlinkDatabaseMetaDataTest.java | 100 + 3 files changed, 447 insertions(+), 408 deletions(-)
[flink] branch release-1.17 updated: [FLINK-32219][sql-client] Fix SqlClient hangs when executing EXECUTE PLAN statement (#22725)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch release-1.17 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.17 by this push: new be53133546d [FLINK-32219][sql-client] Fix SqlClient hangs when executing EXECUTE PLAN statement (#22725) be53133546d is described below commit be53133546d054e282c0a24bfe722d0d276f9a8f Author: Jane Chan <55568005+ladyfor...@users.noreply.github.com> AuthorDate: Thu Jun 8 10:11:17 2023 +0800 [FLINK-32219][sql-client] Fix SqlClient hangs when executing EXECUTE PLAN statement (#22725) (cherry picked from commit 83ba6b5348cbffb26e8d1d5ce6e8d6bb1994e3bc) --- .../service/operation/OperationExecutor.java | 16 + .../gateway/AbstractSqlGatewayStatementITCase.java | 8 +++ .../src/test/resources/sql/insert.q| 83 ++ .../table/api/internal/TableEnvironmentImpl.java | 36 +- 4 files changed, 126 insertions(+), 17 deletions(-) diff --git a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java index 22c4ffcdf32..1be89881aec 100644 --- a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java +++ b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java @@ -67,6 +67,7 @@ import org.apache.flink.table.gateway.service.result.ResultFetcher; import org.apache.flink.table.gateway.service.utils.SqlExecutionException; import org.apache.flink.table.module.ModuleManager; import org.apache.flink.table.operations.BeginStatementSetOperation; +import org.apache.flink.table.operations.CompileAndExecutePlanOperation; import org.apache.flink.table.operations.DeleteFromFilterOperation; import org.apache.flink.table.operations.EndStatementSetOperation; import org.apache.flink.table.operations.LoadModuleOperation; @@ -77,6 +78,7 @@ import org.apache.flink.table.operations.StatementSetOperation; import org.apache.flink.table.operations.UnloadModuleOperation; import org.apache.flink.table.operations.UseOperation; import org.apache.flink.table.operations.command.AddJarOperation; +import org.apache.flink.table.operations.command.ExecutePlanOperation; import org.apache.flink.table.operations.command.RemoveJarOperation; import org.apache.flink.table.operations.command.ResetOperation; import org.apache.flink.table.operations.command.SetOperation; @@ -423,6 +425,9 @@ public class OperationExecutor { } else if (op instanceof ModifyOperation) { return callModifyOperations( tableEnv, handle, Collections.singletonList((ModifyOperation) op)); +} else if (op instanceof CompileAndExecutePlanOperation +|| op instanceof ExecutePlanOperation) { +return callExecuteOperation(tableEnv, handle, op); } else if (op instanceof StatementSetOperation) { return callModifyOperations( tableEnv, handle, ((StatementSetOperation) op).getOperations()); @@ -512,6 +517,17 @@ public class OperationExecutor { return ResultFetcher.fromTableResult(handle, result, false); } +return fetchJobId(result, handle); +} + +private ResultFetcher callExecuteOperation( +TableEnvironmentInternal tableEnv, +OperationHandle handle, +Operation executePlanOperation) { +return fetchJobId(tableEnv.executeInternal(executePlanOperation), handle); +} + +private ResultFetcher fetchJobId(TableResultInternal result, OperationHandle handle) { JobID jobID = result.getJobClient() .orElseThrow( diff --git a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/AbstractSqlGatewayStatementITCase.java b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/AbstractSqlGatewayStatementITCase.java index 0ba90189172..94813359752 100644 --- a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/AbstractSqlGatewayStatementITCase.java +++ b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/AbstractSqlGatewayStatementITCase.java @@ -116,6 +116,9 @@ public abstract class AbstractSqlGatewayStatementITCase extends AbstractTestBase replaceVars.put( "$VAR_STREAMING_PATH2", Files.createDirectory(temporaryFolder.resolve("streaming2")).toFile().getPath()); +replaceVars.put( +"$VAR_STREAMING_PATH3", + Files.createDirectory(temporaryFolder.resolve("streaming3")).toFile().getPath()); repl
[flink] branch master updated (bc2fee44a87 -> 83ba6b5348c)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from bc2fee44a87 [FLINK-31812][api] Transform savepoint path to null if empty in body add 83ba6b5348c [FLINK-32219][sql-client] Fix SqlClient hangs when executing EXECUTE PLAN statement (#22703) No new revisions were added by this update. Summary of changes: .../service/operation/OperationExecutor.java | 16 + .../gateway/AbstractSqlGatewayStatementITCase.java | 8 +++ .../src/test/resources/sql/insert.q| 83 ++ .../table/api/internal/TableEnvironmentImpl.java | 36 +- 4 files changed, 126 insertions(+), 17 deletions(-)
[flink] branch release-1.16 updated (724302faec9 -> c07b50170d0)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git from 724302faec9 [hotfix][docs] Update minor version for 1.16.2 release add c07b50170d0 [FLINK-31967][table-runtime] Fix LAG function get NPE when the input type is not null No new revisions were added by this update. Summary of changes: .../batch/sql/agg/AggregateITCaseBase.scala| 29 .../runtime/stream/sql/AggregateITCase.scala | 32 ++ .../functions/aggregate/LagAggFunction.java| 9 ++ 3 files changed, 70 insertions(+)
[flink] branch release-1.17 updated: [FLINK-31967][table-runtime] Fix LAG function get NPE when the input type is not null #22647
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch release-1.17 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.17 by this push: new 6cac01d827d [FLINK-31967][table-runtime] Fix LAG function get NPE when the input type is not null #22647 6cac01d827d is described below commit 6cac01d827ddb58bd79b5b42e1af4e05fbc45814 Author: whjshj <51083896+whj...@users.noreply.github.com> AuthorDate: Thu May 25 19:55:57 2023 +0800 [FLINK-31967][table-runtime] Fix LAG function get NPE when the input type is not null #22647 Co-authored-by: 两宽 --- .../batch/sql/agg/AggregateITCaseBase.scala| 28 +++ .../runtime/stream/sql/AggregateITCase.scala | 31 ++ .../functions/aggregate/LagAggFunction.java| 9 +++ 3 files changed, 68 insertions(+) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateITCaseBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateITCaseBase.scala index 556f09eba54..5339396b9c8 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateITCaseBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateITCaseBase.scala @@ -1184,6 +1184,34 @@ abstract class AggregateITCaseBase(testName: String) extends BatchTestBase { checkResult("select count(*) from src", Seq(row(3))) } + @Test + def testLeadAggFunction(): Unit = { +val data = + List(rowOf(2L, 15, "Hello"), rowOf(8L, 11, "Hello world"), rowOf(9L, 12, "Hello world!")) +val dataId = TestValuesTableFactory.registerData(data) +tEnv.executeSql(s""" + |CREATE TABLE src( + | `id` BIGINT, + | `len` INT NOT NULL, + | `content` STRING, + | `proctime` AS PROCTIME() + |) WITH ( + | 'connector' = 'values', + | 'bounded' = 'true', + | 'data-id' = '$dataId' + |) + |""".stripMargin) +val sql = + s""" + |select + | LEAD(len) OVER w AS prev_quantity, + | LEAD(len, 1, cast(null as int)) OVER w AS prev_quantity + |from src + |WINDOW w AS (ORDER BY proctime) + |""".stripMargin +checkResult(sql, Seq(row("11, 11"), row("12, 12"), row("null, null"))) + } + // TODO support csv // @Test // def testMultiGroupBys(): Unit = { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateITCase.scala index 1e6cf31ed7b..771373292b9 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateITCase.scala @@ -1765,4 +1765,35 @@ class AggregateITCase(aggMode: AggMode, miniBatch: MiniBatchMode, backend: State val expected = List("3") assertEquals(expected.sorted, sink.getRetractResults.sorted) } + + @Test + def testLagAggFunction(): Unit = { +val data = + List(rowOf(2L, 15, "Hello"), rowOf(8L, 11, "Hello world"), rowOf(9L, 12, "Hello world!")) +val dataId = TestValuesTableFactory.registerData(data) +tEnv.executeSql(s""" + |CREATE TABLE src( + | `id` BIGINT, + | `len` INT NOT NULL, + | `content` STRING, + | `proctime` AS PROCTIME() + |) WITH ( + | 'connector' = 'values', + | 'data-id' = '$dataId' + |) + |""".stripMargin) +val sink = new TestingRetractSink +val sql = + s""" + |select + | LAG(len, 1, cast(null as int)) OVER w AS prev_quantity, + | LAG(len) OVER w AS prev_quantity + |from src + |WINDOW w AS (ORDER BY proctime) + |""".stripMargin +tEnv.sqlQuery(sql).toRetractStream[Row].addSink(sink).setParallelism(1) +env.execute() +val expected = List("
[flink] branch master updated: [FLINK-32043][sql-gateway] Fix Sql Gateway doesn't validate set statement (#22644)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new 1ef847b2c52 [FLINK-32043][sql-gateway] Fix Sql Gateway doesn't validate set statement (#22644) 1ef847b2c52 is described below commit 1ef847b2c5244de0ba351dff3f21701acb8f3cce Author: Shengkai <33114724+fsk...@users.noreply.github.com> AuthorDate: Thu May 25 19:49:58 2023 +0800 [FLINK-32043][sql-gateway] Fix Sql Gateway doesn't validate set statement (#22644) --- .../gateway/service/context/SessionContext.java| 3 +- .../service/operation/OperationExecutor.java | 1 - .../gateway/AbstractSqlGatewayStatementITCase.java | 3 + .../service/context/SessionContextTest.java| 6 -- .../flink-sql-gateway/src/test/resources/sql/set.q | 68 ++ 5 files changed, 73 insertions(+), 8 deletions(-) diff --git a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/context/SessionContext.java b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/context/SessionContext.java index d9dddf20a36..81055fc05b8 100644 --- a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/context/SessionContext.java +++ b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/context/SessionContext.java @@ -135,7 +135,8 @@ public class SessionContext { public void set(String key, String value) { try { // Test whether the key value will influence the creation of the Executor. - createOperationExecutor(Configuration.fromMap(Collections.singletonMap(key, value))); + createOperationExecutor(Configuration.fromMap(Collections.singletonMap(key, value))) +.getTableEnvironment(); } catch (Exception e) { // get error and reset the key with old value throw new SqlExecutionException( diff --git a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java index 6ba1b544500..22c4ffcdf32 100644 --- a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java +++ b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java @@ -301,7 +301,6 @@ public class OperationExecutor { // -@VisibleForTesting public TableEnvironmentInternal getTableEnvironment() { // checks the value of RUNTIME_MODE Configuration operationConfig = sessionContext.getSessionConf().clone(); diff --git a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/AbstractSqlGatewayStatementITCase.java b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/AbstractSqlGatewayStatementITCase.java index 097faac75b3..00ed0f9084a 100644 --- a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/AbstractSqlGatewayStatementITCase.java +++ b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/AbstractSqlGatewayStatementITCase.java @@ -65,6 +65,7 @@ import java.util.jar.JarFile; import java.util.regex.Pattern; import java.util.stream.Collectors; +import static org.apache.flink.configuration.RestOptions.PORT; import static org.apache.flink.table.gateway.utils.SqlScriptReader.HINT_START_OF_OUTPUT; import static org.apache.flink.table.planner.utils.TableTestUtil.replaceNodeIdInOperator; import static org.apache.flink.table.planner.utils.TableTestUtil.replaceStreamNodeId; @@ -121,6 +122,8 @@ public abstract class AbstractSqlGatewayStatementITCase extends AbstractTestBase replaceVars.put( "$VAR_BATCH_CTAS_PATH", Files.createDirectory(temporaryFolder.resolve("batch_ctas")).toFile().getPath()); +replaceVars.put( +"$VAR_REST_PORT", MINI_CLUSTER.getClientConfiguration().get(PORT).toString()); } @TestTemplate diff --git a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/context/SessionContextTest.java b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/context/SessionContextTest.java index f6fdb76a5c7..8d1eb4d6b52 100644 --- a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/context/SessionContextTest.java +++ b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/context/SessionContextTest.java @@ -64,15 +64
[flink] branch master updated (24701ca4edf -> ac6aedbf0f3)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 24701ca4edf [FLINK-32154][build] Setup checkstyle rules to forbid mockito/powermock add ac6aedbf0f3 [FLINK-31967][table-runtime] Fix LAG function get NPE when the input type is not null No new revisions were added by this update. Summary of changes: .../batch/sql/agg/AggregateITCaseBase.scala| 28 +++ .../runtime/stream/sql/AggregateITCase.scala | 31 ++ .../functions/aggregate/LagAggFunction.java| 9 +++ 3 files changed, 68 insertions(+)
[flink] branch release-1.17 updated: [FLINK-32043][sql-gateway] Fix Sql Gateway doesn't validate set state… (#22633)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch release-1.17 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.17 by this push: new 23030f6546a [FLINK-32043][sql-gateway] Fix Sql Gateway doesn't validate set state… (#22633) 23030f6546a is described below commit 23030f6546a5f5877166ee1dc6f49dd18f4dc188 Author: Shengkai <33114724+fsk...@users.noreply.github.com> AuthorDate: Thu May 25 10:04:30 2023 +0800 [FLINK-32043][sql-gateway] Fix Sql Gateway doesn't validate set state… (#22633) * [FLINK-32043][sql-gateway] Fix Sql Gateway doesn't validate set statement --- .../gateway/service/context/SessionContext.java| 3 +- .../service/operation/OperationExecutor.java | 1 - .../gateway/AbstractSqlGatewayStatementITCase.java | 3 + .../service/context/SessionContextTest.java| 6 -- .../flink-sql-gateway/src/test/resources/sql/set.q | 68 ++ 5 files changed, 73 insertions(+), 8 deletions(-) diff --git a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/context/SessionContext.java b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/context/SessionContext.java index d9dddf20a36..81055fc05b8 100644 --- a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/context/SessionContext.java +++ b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/context/SessionContext.java @@ -135,7 +135,8 @@ public class SessionContext { public void set(String key, String value) { try { // Test whether the key value will influence the creation of the Executor. - createOperationExecutor(Configuration.fromMap(Collections.singletonMap(key, value))); + createOperationExecutor(Configuration.fromMap(Collections.singletonMap(key, value))) +.getTableEnvironment(); } catch (Exception e) { // get error and reset the key with old value throw new SqlExecutionException( diff --git a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java index 6ba1b544500..22c4ffcdf32 100644 --- a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java +++ b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java @@ -301,7 +301,6 @@ public class OperationExecutor { // -@VisibleForTesting public TableEnvironmentInternal getTableEnvironment() { // checks the value of RUNTIME_MODE Configuration operationConfig = sessionContext.getSessionConf().clone(); diff --git a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/AbstractSqlGatewayStatementITCase.java b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/AbstractSqlGatewayStatementITCase.java index 890e55acc02..0ba90189172 100644 --- a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/AbstractSqlGatewayStatementITCase.java +++ b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/AbstractSqlGatewayStatementITCase.java @@ -65,6 +65,7 @@ import java.util.jar.JarFile; import java.util.regex.Pattern; import java.util.stream.Collectors; +import static org.apache.flink.configuration.RestOptions.PORT; import static org.apache.flink.table.gateway.utils.SqlScriptReader.HINT_START_OF_OUTPUT; import static org.apache.flink.table.planner.utils.TableTestUtil.replaceNodeIdInOperator; import static org.apache.flink.table.planner.utils.TableTestUtil.replaceStreamNodeId; @@ -121,6 +122,8 @@ public abstract class AbstractSqlGatewayStatementITCase extends AbstractTestBase replaceVars.put( "$VAR_BATCH_CTAS_PATH", Files.createDirectory(temporaryFolder.resolve("batch_ctas")).toFile().getPath()); +replaceVars.put( +"$VAR_REST_PORT", MINI_CLUSTER.getClientConfiguration().get(PORT).toString()); } @TestTemplate diff --git a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/context/SessionContextTest.java b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/context/SessionContextTest.java index bc6976b887c..8115379b7c0 100644 --- a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/context/SessionContextTest.java +++ b/flink-table/flink-sql-gateway/src/test/
[flink] branch master updated (742685b76c7 -> 333e023196d)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 742685b76c7 [FLINK-30921][ci] Adds mirrors instead of relying on a single source for Ubuntu packages add 333e023196d [FLINK-31917][table-planner] Fix the idempotence lost in JsonSerDe round trip for AggregateCall and RexNode (#22478) No new revisions were added by this update. Summary of changes: .../exec/serde/AggregateCallJsonDeserializer.java | 4 +- .../nodes/exec/serde/RexNodeJsonSerializer.java| 39 ++--- .../nodes/exec/serde/AggregateCallSerdeTest.java | 158 + .../plan/nodes/exec/serde/JsonSerdeTestUtil.java | 1 + .../nodes/exec/serde/RexNodeJsonSerdeTest.java | 11 +- .../flink/table/planner/utils/JsonTestUtils.java | 5 + .../CalcJsonPlanTest_jsonplan/testComplexCalc.out | 14 +- .../CalcJsonPlanTest_jsonplan/testSimpleFilter.out | 4 +- .../ExpandJsonPlanTest_jsonplan/testExpand.out | 14 +- ...tDistinctAggCalls[isMiniBatchEnabled=false].out | 4 +- ...stDistinctAggCalls[isMiniBatchEnabled=true].out | 4 +- ...gCallsWithGroupBy[isMiniBatchEnabled=false].out | 4 +- ...ggCallsWithGroupBy[isMiniBatchEnabled=true].out | 4 +- ...AggWithoutGroupBy[isMiniBatchEnabled=false].out | 4 +- ...eAggWithoutGroupBy[isMiniBatchEnabled=true].out | 4 +- ...erDefinedAggCalls[isMiniBatchEnabled=false].out | 6 +- ...serDefinedAggCalls[isMiniBatchEnabled=true].out | 6 +- .../testIncrementalAggregate.out | 4 +- ...lAggregateWithSumCountDistinctAndRetraction.out | 4 +- .../testMatch.out | 8 +- .../testProcTimeBoundedPartitionedRangeOver.out| 4 +- ...undedPartitionedRowsOverWithBuiltinProctime.out | 6 +- .../testProcTimeUnboundedPartitionedRangeOver.out | 4 +- ...stProctimeBoundedDistinctPartitionedRowOver.out | 6 +- ...edDistinctWithNonDistinctPartitionedRowOver.out | 8 +- .../testRowTimeBoundedPartitionedRowsOver.out | 4 +- .../testPythonFunctionInWhereClause.out| 4 +- .../testJoinWithFilter.out | 4 +- .../tesPythonAggCallsWithGroupBy.out | 4 +- .../testEventTimeHopWindow.out | 4 +- .../testEventTimeSessionWindow.out | 4 +- .../testEventTimeTumbleWindow.out | 4 +- .../testProcTimeHopWindow.out | 4 +- .../testProcTimeSessionWindow.out | 4 +- .../testProcTimeTumbleWindow.out | 4 +- ...undedPartitionedRowsOverWithBuiltinProctime.out | 4 +- .../testRowTimeBoundedPartitionedRowsOver.out | 4 +- .../testFilterPushDown.out | 4 +- .../ValuesJsonPlanTest_jsonplan/testValues.out | 10 +- .../testDistinctSplitEnabled.out | 4 +- .../testFollowedByWindowJoin.out | 6 +- .../flink/table/planner/utils/TableTestBase.scala | 19 ++- 42 files changed, 295 insertions(+), 126 deletions(-) create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/AggregateCallSerdeTest.java
[flink] branch master updated: [FLINK-31621][table] Add built-in ARRAY_REVERSE function. (#22277)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new 122429f99b9 [FLINK-31621][table] Add built-in ARRAY_REVERSE function. (#22277) 122429f99b9 is described below commit 122429f99b9b75737cd218a56a37a7ced750582f Author: Jacky Lau AuthorDate: Fri Mar 31 19:10:24 2023 +0800 [FLINK-31621][table] Add built-in ARRAY_REVERSE function. (#22277) --- docs/data/sql_functions.yml| 3 ++ .../docs/reference/pyflink.table/expressions.rst | 1 + flink-python/pyflink/table/expression.py | 7 +++ .../flink/table/api/internal/BaseExpressions.java | 10 .../functions/BuiltInFunctionDefinitions.java | 13 + .../functions/CollectionFunctionsITCase.java | 41 ++- .../functions/scalar/ArrayDistinctFunction.java| 2 +- .../functions/scalar/ArrayRemoveFunction.java | 2 +- .../functions/scalar/ArrayReverseFunction.java | 58 ++ 9 files changed, 134 insertions(+), 3 deletions(-) diff --git a/docs/data/sql_functions.yml b/docs/data/sql_functions.yml index 81119ce214f..5a842a2ac63 100644 --- a/docs/data/sql_functions.yml +++ b/docs/data/sql_functions.yml @@ -640,6 +640,9 @@ collection: - sql: ARRAY_REMOVE(haystack, needle) table: haystack.arrayRemove(needle) description: Removes all elements that equal to element from array. If the array itself is null, the function will return null. Keeps ordering of elements. + - sql: ARRAY_REVERSE(haystack) +table: haystack.arrayReverse() +description: Returns an array in reverse order. If the array itself is null, the function will return null. - sql: MAP_KEYS(map) table: MAP.mapKeys() description: Returns the keys of the map as array. No order guaranteed. diff --git a/flink-python/docs/reference/pyflink.table/expressions.rst b/flink-python/docs/reference/pyflink.table/expressions.rst index ec86f6a9887..117f825a00e 100644 --- a/flink-python/docs/reference/pyflink.table/expressions.rst +++ b/flink-python/docs/reference/pyflink.table/expressions.rst @@ -229,6 +229,7 @@ advanced type helper functions Expression.array_distinct Expression.array_position Expression.array_remove +Expression.array_reverse Expression.map_keys Expression.map_values diff --git a/flink-python/pyflink/table/expression.py b/flink-python/pyflink/table/expression.py index 99797ecc83b..f40824da492 100644 --- a/flink-python/pyflink/table/expression.py +++ b/flink-python/pyflink/table/expression.py @@ -1505,6 +1505,13 @@ class Expression(Generic[T]): """ return _binary_op("arrayRemove")(self, needle) +def array_reverse(self) -> 'Expression': +""" +Returns an array in reverse order. +If the array itself is null, the function will return null. +""" +return _binary_op("arrayReverse")(self) + @property def map_keys(self) -> 'Expression': """ diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java index 0d643087b7f..0428a4689f8 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java @@ -58,6 +58,7 @@ import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.ARRAY_ import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.ARRAY_ELEMENT; import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.ARRAY_POSITION; import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.ARRAY_REMOVE; +import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.ARRAY_REVERSE; import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.ASCII; import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.ASIN; import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.AT; @@ -1386,6 +1387,15 @@ public abstract class BaseExpressions { unresolvedCall(ARRAY_REMOVE, toExpr(), objectToExpression(needle))); } +/** + * Returns an array in reverse order. + * + * If the array itself is null, the function will return null. + */ +public OutType arrayReverse() { +return toApiSpecificExpression(unresolvedCall(ARRAY_REVERSE, toExpr())); +} + /** Returns the keys of the map as an array. */ public OutType mapKeys() { return toApiSpecificExpression(unresolvedCa
[flink] branch master updated: [FLINK-31554][docs-zh] Translate "Overview" page of "sql-gateway" into Chinese (#22235)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new 6620641db32 [FLINK-31554][docs-zh] Translate "Overview" page of "sql-gateway" into Chinese (#22235) 6620641db32 is described below commit 6620641db32a95feca12c1bbebdde1bdc1ee9056 Author: Shammon FY AuthorDate: Mon Mar 27 09:38:12 2023 +0800 [FLINK-31554][docs-zh] Translate "Overview" page of "sql-gateway" into Chinese (#22235) --- .../docs/dev/table/sql-gateway/overview.md | 74 +++--- 1 file changed, 36 insertions(+), 38 deletions(-) diff --git a/docs/content.zh/docs/dev/table/sql-gateway/overview.md b/docs/content.zh/docs/dev/table/sql-gateway/overview.md index 2bb3ea69cb1..81b1bea869f 100644 --- a/docs/content.zh/docs/dev/table/sql-gateway/overview.md +++ b/docs/content.zh/docs/dev/table/sql-gateway/overview.md @@ -1,5 +1,5 @@ --- -title: Overview +title: 概览 weight: 1 type: docs aliases: @@ -24,47 +24,45 @@ specific language governing permissions and limitations under the License. --> -Introduction +介绍 -The SQL Gateway is a service that enables multiple clients from the remote to execute SQL in concurrency. It provides -an easy way to submit the Flink Job, look up the metadata, and analyze the data online. +SQL Gateway 服务支持并发执行从多个client提交的 SQL。它提供了一种简单的方法来提交 Flink 作业、查找元数据和在线分析数据。 -The SQL Gateway is composed of pluggable endpoints and the `SqlGatewayService`. The `SqlGatewayService` is a processor that is -reused by the endpoints to handle the requests. The endpoint is an entry point that allows users to connect. Depending on the -type of the endpoints, users can use different utils to connect. +SQL Gateway 由插件化的 endpoint 和 `SqlGatewayService` 组成。多个 endpoint 可以复用 `SqlGatewayService` 处理请求。endpoint 是用户连接的入口。 +用户可以使用不同的工具连接不同类型的 endpoint。 {{< img width="80%" src="/fig/sql-gateway-architecture.png" alt="SQL Gateway Architecture" >}} -Getting Started +开始 --- -This section describes how to setup and run your first Flink SQL program from the command-line. - -The SQL Gateway is bundled in the regular Flink distribution and thus runnable out-of-the-box. It requires only a running Flink cluster where table programs can be executed. For more information about setting up a Flink cluster see the [Cluster & Deployment]({{< ref "docs/deployment/resource-providers/standalone/overview" >}}) part. If you simply want to try out the SQL Client, you can also start a local cluster with one worker using the following command: +这个章节描述如何通过命令行启动和执行你的第一个 Flink SQL 作业。 +SQL Gateway 和 Flink 版本一起发布,开箱即用。它只需要一个正在运行的 Flink 集群,可以执行 Flink SQL 作业。 +更多启动 Flink 集群的信息可以查看 [Cluster & Deployment]({{< ref "docs/deployment/resource-providers/standalone/overview" >}})。 +如果你只是想简单尝试 SQL Client,你也可以使用以下命令启动只有一个 worker 的本地集群。 ```bash $ ./bin/start-cluster.sh ``` ### Starting the SQL Gateway -The SQL Gateway scripts are also located in the binary directory of Flink. Users can start by calling: +SQL Gateway 脚本也在 Flink 二进制包的目录中。用户通过以下命令启动: ```bash $ ./bin/sql-gateway.sh start -Dsql-gateway.endpoint.rest.address=localhost ``` -The command starts the SQL Gateway with REST Endpoint that listens on the address localhost:8083. You can use the curl command to check -whether the REST Endpoint is available. +这个命令启动 SQL Gateway 和 REST Endpoint,监听 localhost:8083 地址。你可以使用 curl 命令检查 REST Endpoint 是否存活。 ```bash $ curl http://localhost:8083/v1/info {"productName":"Apache Flink","version":"{{< version >}}"} ``` -### Running SQL Queries +### 执行 SQL 查询 -For validating your setup and cluster connection, you can work with following steps. +你可以通过以下步骤来验证集群配置和连接。 **Step 1: Open a session** @@ -73,7 +71,7 @@ $ curl --request POST http://localhost:8083/v1/sessions {"sessionHandle":"..."} ``` -The `sessionHandle` in the return results is used by the SQL Gateway to uniquely identify every active user. +SQL Gateway 返回结果中的 `sessionHandle` 用来唯一标识每个活跃用户。 **Step 2: Execute a query** @@ -82,12 +80,12 @@ $ curl --request POST http://localhost:8083/v1/sessions/${sessionHandle}/stateme {"operationHandle":"..."} ``` -The `operationHandle` in the return results is used by the SQL Gateway to uniquely identify the submitted SQL. +SQL Gateway 返回结果中的 `operationHandle` 用来唯一标识提交的 SQL。 **Step 3: Fetch results** -With the `sessionHandle` and `operationHandle` above, you can fetch the corresponding results. +通过上述 `sessionHandle` 和 `operationHandle`,你能获取相应的结果。 ```bash $ curl --request GET http://localhost:8083/v1/sessions/${sessionHandle}/operations/${operationHandle}/result/0 @@ -116,18
[flink] branch master updated: [FLINK-31556][docs-zh] Translate "rest" page of "sql-gateway" into Chinese (#22237)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new fc38c923f68 [FLINK-31556][docs-zh] Translate "rest" page of "sql-gateway" into Chinese (#22237) fc38c923f68 is described below commit fc38c923f68740a21013daea6938cf772dc3e08a Author: Shammon FY AuthorDate: Mon Mar 27 09:36:28 2023 +0800 [FLINK-31556][docs-zh] Translate "rest" page of "sql-gateway" into Chinese (#22237) --- docs/content.zh/docs/dev/table/sql-gateway/rest.md | 43 ++ 1 file changed, 20 insertions(+), 23 deletions(-) diff --git a/docs/content.zh/docs/dev/table/sql-gateway/rest.md b/docs/content.zh/docs/dev/table/sql-gateway/rest.md index 3d9d6152300..5caea340bd4 100644 --- a/docs/content.zh/docs/dev/table/sql-gateway/rest.md +++ b/docs/content.zh/docs/dev/table/sql-gateway/rest.md @@ -26,28 +26,25 @@ under the License. # REST Endpoint -The REST endpoint allows user to connect to SQL Gateway with REST API. +REST endpoint 允许用户通过 REST API 连接 SQL Gateway。 Overview of SQL Processing ### Open Session -When the client connects to the SQL Gateway, the SQL Gateway creates a `Session` as the context to store the users-specified information -during the interactions between the client and SQL Gateway. After the creation of the `Session`, the SQL Gateway server returns an identifier named -`SessionHandle` for later interactions. +当客户端连接到 SQL Gateway 时,SQL Gateway 会创建一个 `Session`,存储客户端和 SQL Gateway 交互期间的用户相关信息。 +创建 `Session` 后,SQL Gateway 会返回 `SessionHandle` 标识,用于后续的交互。 ### Submit SQL -After the registration of the `Session`, the client can submit the SQL to the SQL Gateway server. When submitting the SQL, -the SQL is translated to an `Operation` and an identifier named `OperationHandle` is returned for fetch results later. The Operation has -its lifecycle, the client is able to cancel the execution of the `Operation` or close the `Operation` to release the resources used by the `Operation`. +注册 `Session` 后,客户端能够提交 SQL 到 SQL Gateway。提交 SQL 后,SQL 会被转换成 `Operation`,并且返回 `OperationHandle` 标识,用于用户后续获取结果。 +Operation 有它的生命周期,客户端可以取消正在执行的 `Operation`,或者关闭 `Operation` 并释放它使用的资源。 ### Fetch Results -With the `OperationHandle`, the client can fetch the results from the `Operation`. If the `Operation` is ready, the SQL Gateway will return a batch -of the data with the corresponding schema and a URI that is used to fetch the next batch of the data. When all results have been fetched, the -SQL Gateway will fill the `resultType` in the response with value `EOS` and the URI to the next batch of the data is null. +客户端可以通过 `OperationHandle` 从 `Operation` 获取结果。当一个 `Operation` 已经就绪,SQL Gateway 将返回一个包含对应 schema 和 URI 的批式数据, +URI 可以被用来获取下一个批式数据。当所有结果已经获取完成,SQL Gateway 会将结果中的 `resultType` 设置为 `EOS`,并且将获取下一个批式数据的 URI 设置为 null。 {{< img width="100%" src="/fig/sql-gateway-interactions.png" alt="SQL Gateway Interactions" >}} @@ -68,25 +65,25 @@ Endpoint Options sql-gateway.endpoint.rest.address (none) String -The address that should be used by clients to connect to the sql gateway server. +客户端通过该地址连接到 SQL Gateway 服务。 sql-gateway.endpoint.rest.bind-address (none) String -The address that the sql gateway server binds itself. +SQL Gateway 服务绑定的地址。 sql-gateway.endpoint.rest.bind-port "8083" String -The port that the sql gateway server binds itself. Accepts a list of ports (“50100,50101”), ranges (“50100-50200”) or a combination of both. It is recommended to set a range of ports to avoid collisions when multiple sql gateway servers are running on the same machine. +SQL Gateway 服务绑定的端口号。接受端口列表 (“50100,50101”)或端口区间(“50100-50200”),也可以两种方式混用。为了避免同一台机器上运行多个 SQL Gateway 服务引起的端口冲突,建议设置为端口区间。 sql-gateway.endpoint.rest.port 8083 Integer -The port that the client connects to. If bind-port has not been specified, then the sql gateway server will bind to this port. +客户端连接的端口号。如果 bind-port 没有被指定,SQL Gateway 服务将会绑定这个端口。 @@ -94,15 +91,15 @@ Endpoint Options REST API -The available OpenAPI specification is as follows. The default version is v2. +OpenAPI 规范如下,默认版本是 v2。 -| Version | Description | -| --- | --- | -| [Open API v1 specification]({{< ref_static "generated/rest_v1_sql_gateway.yml" >}}) | Allow users to submit statements to the gateway and execute. | -| [Open API v2 specification]({{< ref_stati
[flink] branch master updated (a64781b1ef8 -> c9de75052ea)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from a64781b1ef8 [FLINK-31574][table-planner] Cleanup unused private methods in OperationConverterUtils (#22240) add c9de75052ea [FLINK-31555][docs-zh] Translate "hiveserver2" page of "sql-gateway" into Chinese No new revisions were added by this update. Summary of changes: docs/content.zh/docs/dev/table/sql-gateway/hiveserver2.md | 9 - 1 file changed, 4 insertions(+), 5 deletions(-)
[flink] branch release-1.16 updated (49d9ea6e5d3 -> 6fd3b9b3384)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git from 49d9ea6e5d3 [FLINK-31348][Docs] Fix unclosed shortcodes (/tabs) in the docs, that causes issues with Hugo version 0.111.0 and higher. This closes #22113 add 6fd3b9b3384 [FLINK-31351][sql-gateway] Don't stop the stuck thread by force No new revisions were added by this update. Summary of changes: .../service/operation/OperationManager.java| 69 ++ .../gateway/service/utils/SqlCancelException.java | 12 ++-- .../gateway/service/SqlGatewayServiceITCase.java | 60 +++ 3 files changed, 111 insertions(+), 30 deletions(-) copy flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityMapper.java => flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/utils/SqlCancelException.java (78%)
[flink] branch master updated: [FLINK-31351][sql-gateway] Don't stop the stuck thread by force
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new 384d6b10a2d [FLINK-31351][sql-gateway] Don't stop the stuck thread by force 384d6b10a2d is described below commit 384d6b10a2d69b9384052c3d4c3ad82babd201d1 Author: Shengkai <1059623...@qq.com> AuthorDate: Wed Mar 8 12:12:05 2023 +0800 [FLINK-31351][sql-gateway] Don't stop the stuck thread by force This closes #22127 --- .../service/operation/OperationManager.java| 69 -- .../gateway/service/utils/SqlCancelException.java | 29 .../service/operation/OperationManagerTest.java| 81 +++--- 3 files changed, 131 insertions(+), 48 deletions(-) diff --git a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationManager.java b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationManager.java index 8b239abc771..37a9363c0f2 100644 --- a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationManager.java +++ b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationManager.java @@ -30,7 +30,11 @@ import org.apache.flink.table.gateway.api.results.ResultSet; import org.apache.flink.table.gateway.api.utils.SqlGatewayException; import org.apache.flink.table.gateway.service.result.NotReadyResult; import org.apache.flink.table.gateway.service.result.ResultFetcher; +import org.apache.flink.table.gateway.service.utils.SqlCancelException; import org.apache.flink.table.gateway.service.utils.SqlExecutionException; +import org.apache.flink.util.IOUtils; + +import org.apache.flink.shaded.guava30.com.google.common.util.concurrent.Uninterruptibles; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,6 +48,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.FutureTask; import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -183,13 +188,14 @@ public class OperationManager { /** Closes the {@link OperationManager} and all operations. */ public void close() { stateLock.writeLock().lock(); +Exception closeException = null; try { isRunning = false; -for (Operation operation : submittedOperations.values()) { -operation.close(); -} -submittedOperations.clear(); +IOUtils.closeAll(submittedOperations.values(), Throwable.class); +} catch (Exception e) { +closeException = e; } finally { +submittedOperations.clear(); stateLock.writeLock().unlock(); } // wait all operations closed @@ -201,13 +207,19 @@ public class OperationManager { operationLock.release(); } LOG.debug("Closes the Operation Manager."); +if (closeException != null) { +throw new SqlExecutionException( +"Failed to close the OperationManager.", closeException); +} } // --- /** Operation to manage the execution, results and so on. */ @VisibleForTesting -public class Operation { +public class Operation implements AutoCloseable { + +private static final long WAIT_CLEAN_UP_MILLISECONDS = 5_000; private final OperationHandle operationHandle; @@ -387,7 +399,7 @@ public class OperationManager { private void closeResources() { if (invocation != null && !invocation.isDone()) { invocation.cancel(true); -stopExecutionByForce(invocation); +waitTaskCleanup(invocation); LOG.debug(String.format("Cancel the operation %s.", operationHandle)); } @@ -405,32 +417,23 @@ public class OperationManager { updateState(OperationStatus.ERROR); } -private void stopExecutionByForce(FutureTask invocation) { +private void waitTaskCleanup(FutureTask invocation) { // thread is cleaned async, waiting for a while -Deadline deadline = Deadline.fromNow(Duration.ofSeconds(1)); +Deadline deadline = Deadline.fromNow(Duration.ofMillis(WAIT_CLEAN_UP_MILLISECONDS)); while (deadline.hasTimeLeft()) { Optional threadOptional = getThreadInFuture(invocation); if (!threadOpti
[flink] branch release-1.17 updated: [FLINK-31351][sql-gateway] Don't stop the stuck thread by force
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch release-1.17 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.17 by this push: new 32b37018185 [FLINK-31351][sql-gateway] Don't stop the stuck thread by force 32b37018185 is described below commit 32b370181853f4129fd237c6a57491863a7e8b8c Author: Shengkai <1059623...@qq.com> AuthorDate: Wed Mar 8 12:12:05 2023 +0800 [FLINK-31351][sql-gateway] Don't stop the stuck thread by force This closes #22127 --- .../service/operation/OperationManager.java| 69 -- .../gateway/service/utils/SqlCancelException.java | 29 .../service/operation/OperationManagerTest.java| 81 +++--- 3 files changed, 131 insertions(+), 48 deletions(-) diff --git a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationManager.java b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationManager.java index 8b239abc771..37a9363c0f2 100644 --- a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationManager.java +++ b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationManager.java @@ -30,7 +30,11 @@ import org.apache.flink.table.gateway.api.results.ResultSet; import org.apache.flink.table.gateway.api.utils.SqlGatewayException; import org.apache.flink.table.gateway.service.result.NotReadyResult; import org.apache.flink.table.gateway.service.result.ResultFetcher; +import org.apache.flink.table.gateway.service.utils.SqlCancelException; import org.apache.flink.table.gateway.service.utils.SqlExecutionException; +import org.apache.flink.util.IOUtils; + +import org.apache.flink.shaded.guava30.com.google.common.util.concurrent.Uninterruptibles; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,6 +48,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.FutureTask; import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -183,13 +188,14 @@ public class OperationManager { /** Closes the {@link OperationManager} and all operations. */ public void close() { stateLock.writeLock().lock(); +Exception closeException = null; try { isRunning = false; -for (Operation operation : submittedOperations.values()) { -operation.close(); -} -submittedOperations.clear(); +IOUtils.closeAll(submittedOperations.values(), Throwable.class); +} catch (Exception e) { +closeException = e; } finally { +submittedOperations.clear(); stateLock.writeLock().unlock(); } // wait all operations closed @@ -201,13 +207,19 @@ public class OperationManager { operationLock.release(); } LOG.debug("Closes the Operation Manager."); +if (closeException != null) { +throw new SqlExecutionException( +"Failed to close the OperationManager.", closeException); +} } // --- /** Operation to manage the execution, results and so on. */ @VisibleForTesting -public class Operation { +public class Operation implements AutoCloseable { + +private static final long WAIT_CLEAN_UP_MILLISECONDS = 5_000; private final OperationHandle operationHandle; @@ -387,7 +399,7 @@ public class OperationManager { private void closeResources() { if (invocation != null && !invocation.isDone()) { invocation.cancel(true); -stopExecutionByForce(invocation); +waitTaskCleanup(invocation); LOG.debug(String.format("Cancel the operation %s.", operationHandle)); } @@ -405,32 +417,23 @@ public class OperationManager { updateState(OperationStatus.ERROR); } -private void stopExecutionByForce(FutureTask invocation) { +private void waitTaskCleanup(FutureTask invocation) { // thread is cleaned async, waiting for a while -Deadline deadline = Deadline.fromNow(Duration.ofSeconds(1)); +Deadline deadline = Deadline.fromNow(Duration.ofMillis(WAIT_CLEAN_UP_MILLISECONDS)); while (deadline.hasTimeLeft()) { Optional threadOptional = getThreadInFuture(invoc
[flink] branch release-1.16 updated (cf04b2c08fa -> 93cd23c5543)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git from cf04b2c08fa [FLINK-31133][tests] Prevent timeouts in PartiallyFinishedSourcesITCase add 2dcd6cc6af0 [FLINK-31092][sql-gateway] Fix OperationManager can not kill the running task by force add 93cd23c5543 [FLINK-31092][table-common] Fix ServiceLoaderUtil keeps loading even though the classloader has been closed No new revisions were added by this update. Summary of changes: .../flink/table/gateway/api/utils/ThreadUtils.java | 4 +- .../service/operation/OperationManager.java| 47 +++- .../gateway/service/SqlGatewayServiceITCase.java | 16 + .../apache/flink/table/factories/FactoryUtil.java | 52 -- .../flink/table/factories/ServiceLoaderUtil.java | 83 -- .../flink/table/factories/FactoryUtilTest.java | 20 ++ 6 files changed, 115 insertions(+), 107 deletions(-) delete mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/ServiceLoaderUtil.java
[flink] branch master updated (d5a66063b29 -> f47b3704867)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from d5a66063b29 [FLINK-31324][connector] Reintroduce previous SingleThreadFetcherManager constructor and mark it as deprecated add d2296422933 [FLINK-31092][sql-gateway] Fix OperationManager can not kill the running task by force add f47b3704867 [FLINK-31092][table-common] Fix ServiceLoaderUtil keeps loading even though the classloader has been closed No new revisions were added by this update. Summary of changes: .../flink/table/gateway/api/utils/ThreadUtils.java | 4 +- .../service/operation/OperationManager.java| 47 +++- .../service/operation/OperationManagerTest.java| 29 .../apache/flink/table/factories/FactoryUtil.java | 52 -- .../flink/table/factories/ServiceLoaderUtil.java | 83 -- .../flink/table/factories/FactoryUtilTest.java | 20 ++ 6 files changed, 128 insertions(+), 107 deletions(-) delete mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/ServiceLoaderUtil.java
[flink] branch release-1.17 updated: [FLINK-30978][sql-client] Fix ExecutorImpl#testInterruptException hangs (#22099)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch release-1.17 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.17 by this push: new 5dddc0dba2b [FLINK-30978][sql-client] Fix ExecutorImpl#testInterruptException hangs (#22099) 5dddc0dba2b is described below commit 5dddc0dba2be20806e67769314eecadf56b87a53 Author: Shengkai <33114724+fsk...@users.noreply.github.com> AuthorDate: Mon Mar 6 18:15:35 2023 +0800 [FLINK-30978][sql-client] Fix ExecutorImpl#testInterruptException hangs (#22099) --- .../flink/table/client/gateway/ExecutorImpl.java | 58 -- .../table/client/gateway/ExecutorImplITCase.java | 19 --- 2 files changed, 46 insertions(+), 31 deletions(-) diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/ExecutorImpl.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/ExecutorImpl.java index 6d9a5206f2b..08b68319860 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/ExecutorImpl.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/ExecutorImpl.java @@ -205,32 +205,40 @@ public class ExecutorImpl implements Executor { ExecuteStatementHeaders.getInstance(), new SessionMessageParameters(sessionHandle), request); + // It's possible that the execution is canceled during the submission. // Close the Operation in background to make sure the execution can continue. -getResponse( -executeStatementResponse, -e -> { -executorService.submit( -() -> { -try { -ExecuteStatementResponseBody executeStatementResponseBody = -executeStatementResponse.get(); -// close operation in background to make sure users can not -// interrupt the execution. -closeOperationAsync( -getOperationHandle( - executeStatementResponseBody - ::getOperationHandle)); -} catch (Exception newException) { -// ignore -} -}); -return new SqlExecutionException("Interrupted to get response.", e); -}); - OperationHandle operationHandle = getOperationHandle( -() -> getResponse(executeStatementResponse).getOperationHandle()); +() -> +getResponse( +executeStatementResponse, +e -> { +executorService.submit( +() -> { +try { + ExecuteStatementResponseBody + executeStatementResponseBody = + executeStatementResponse + .get(); +// close operation in background +// to make sure users can not +// interrupt the execution. + closeOperationAsync( + getOperationHandle( + executeStatementResponseBody + ::getOperationHandle)); +} catch (Exception newException) { + e.addSuppressed(newException); +LOG.error( + &quo
[flink] branch master updated (8d52415a05b -> d96bb2f66d7)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 8d52415a05b [FLINK-31185][python] Support side-output in broadcast processing add d96bb2f66d7 [FLINK-30948][sql-client] Fix ExecutorImpl#testInterruptException hangs (#22055) No new revisions were added by this update. Summary of changes: .../flink/table/client/gateway/ExecutorImpl.java | 58 -- .../table/client/gateway/ExecutorImplITCase.java | 19 --- 2 files changed, 46 insertions(+), 31 deletions(-)
[flink] 02/02: [FLINK-31092][table-common] Fix ServiceLoaderUtil keeps loading even though the classloader has been closed
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch release-1.17 in repository https://gitbox.apache.org/repos/asf/flink.git commit 86e12eb3fcec54d234154e59f8cb0557fc616494 Author: Shengkai <1059623...@qq.com> AuthorDate: Thu Mar 2 20:05:51 2023 +0800 [FLINK-31092][table-common] Fix ServiceLoaderUtil keeps loading even though the classloader has been closed This closes #22072 Co-authored-by: Matthias Pohl --- .../apache/flink/table/factories/FactoryUtil.java | 52 -- .../flink/table/factories/ServiceLoaderUtil.java | 83 -- .../flink/table/factories/FactoryUtilTest.java | 20 ++ 3 files changed, 50 insertions(+), 105 deletions(-) diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/FactoryUtil.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/FactoryUtil.java index d90dbc6c2f8..e22ba4051da 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/FactoryUtil.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/FactoryUtil.java @@ -52,11 +52,12 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedList; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.ServiceLoader; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -801,27 +802,34 @@ public final class FactoryUtil { } static List discoverFactories(ClassLoader classLoader) { -final List result = new LinkedList<>(); -ServiceLoaderUtil.load(Factory.class, classLoader) -.forEach( -loadResult -> { -if (loadResult.hasFailed()) { -if (loadResult.getError() instanceof NoClassDefFoundError) { -LOG.debug( -"NoClassDefFoundError when loading a " -+ Factory.class -+ ". This is expected when trying to load a format dependency but no flink-connector-files is loaded.", -loadResult.getError()); -// After logging, we just ignore this failure -return; -} -throw new TableException( -"Unexpected error when trying to load service provider for factories.", -loadResult.getError()); -} -result.add(loadResult.getService()); -}); -return result; +final Iterator serviceLoaderIterator = +ServiceLoader.load(Factory.class, classLoader).iterator(); + +final List loadResults = new ArrayList<>(); +while (true) { +try { +// error handling should also be applied to the hasNext() call because service +// loading might cause problems here as well +if (!serviceLoaderIterator.hasNext()) { +break; +} + +loadResults.add(serviceLoaderIterator.next()); +} catch (Throwable t) { +if (t instanceof NoClassDefFoundError) { +LOG.debug( +"NoClassDefFoundError when loading a " ++ Factory.class.getCanonicalName() ++ ". This is expected when trying to load a format dependency but no flink-connector-files is loaded.", +t); +} else { +throw new TableException( +"Unexpected error when trying to load service provider.", t); +} +} +} + +return loadResults; } private static String stringifyOption(String key, String value) { diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/ServiceLoaderUtil.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/ServiceLoaderUtil.java deleted file mode 100644 index 620e9c3230a..000 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/ServiceLoaderUtil.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NO
[flink] 01/02: [FLINK-31092][sql-gateway] Fix OperationManager can not kill the running task by force
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch release-1.17 in repository https://gitbox.apache.org/repos/asf/flink.git commit 594010624f8084efd99d6d405b5310ab24013aeb Author: Shengkai <1059623...@qq.com> AuthorDate: Thu Mar 2 17:57:55 2023 +0800 [FLINK-31092][sql-gateway] Fix OperationManager can not kill the running task by force --- .../flink/table/gateway/api/utils/ThreadUtils.java | 4 +- .../service/operation/OperationManager.java| 47 +- .../service/operation/OperationManagerTest.java| 29 + 3 files changed, 78 insertions(+), 2 deletions(-) diff --git a/flink-table/flink-sql-gateway-api/src/main/java/org/apache/flink/table/gateway/api/utils/ThreadUtils.java b/flink-table/flink-sql-gateway-api/src/main/java/org/apache/flink/table/gateway/api/utils/ThreadUtils.java index 330d8ed9eb9..9408c61ecbf 100644 --- a/flink-table/flink-sql-gateway-api/src/main/java/org/apache/flink/table/gateway/api/utils/ThreadUtils.java +++ b/flink-table/flink-sql-gateway-api/src/main/java/org/apache/flink/table/gateway/api/utils/ThreadUtils.java @@ -19,6 +19,7 @@ package org.apache.flink.table.gateway.api.utils; import org.apache.flink.annotation.Internal; +import org.apache.flink.util.concurrent.ExecutorThreadFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,6 +48,7 @@ public class ThreadUtils { poolQueueSize, keepAliveMs, TimeUnit.MILLISECONDS, -new SynchronousQueue<>()); +new SynchronousQueue<>(), +new ExecutorThreadFactory(threadPoolName)); } } diff --git a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationManager.java b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationManager.java index 4e16cb5dd6c..8b239abc771 100644 --- a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationManager.java +++ b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationManager.java @@ -20,6 +20,7 @@ package org.apache.flink.table.gateway.service.operation; import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.time.Deadline; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.gateway.api.operation.OperationHandle; import org.apache.flink.table.gateway.api.operation.OperationStatus; @@ -34,8 +35,11 @@ import org.apache.flink.table.gateway.service.utils.SqlExecutionException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.lang.reflect.Field; +import java.time.Duration; import java.util.HashMap; import java.util.Map; +import java.util.Optional; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.FutureTask; @@ -366,7 +370,6 @@ public class OperationManager { String.format( "Failed to convert the Operation Status from %s to %s for %s.", currentStatus, toStatus, operationHandle); -LOG.error(message); throw new SqlGatewayException(message); } } while (!status.compareAndSet(currentStatus, toStatus)); @@ -384,6 +387,7 @@ public class OperationManager { private void closeResources() { if (invocation != null && !invocation.isDone()) { invocation.cancel(true); +stopExecutionByForce(invocation); LOG.debug(String.format("Cancel the operation %s.", operationHandle)); } @@ -400,6 +404,47 @@ public class OperationManager { // when status is error. updateState(OperationStatus.ERROR); } + +private void stopExecutionByForce(FutureTask invocation) { +// thread is cleaned async, waiting for a while +Deadline deadline = Deadline.fromNow(Duration.ofSeconds(1)); +while (deadline.hasTimeLeft()) { +Optional threadOptional = getThreadInFuture(invocation); +if (!threadOptional.isPresent()) { +// thread has been cleaned up +return; +} +} +Optional threadOptional = getThreadInFuture(invocation); +if (threadOptional.isPresent()) { +// we have to use Thread.stop() here, because this can +// guarantee thread to be stopped, even there is some +// potential consistent problem, we are fine with it. +Thread thread = t
[flink] branch release-1.17 updated (7dd61c31714 -> 86e12eb3fce)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch release-1.17 in repository https://gitbox.apache.org/repos/asf/flink.git from 7dd61c31714 [FLINK-31288][doc] Update doc ralated to overdraft buffer. new 594010624f8 [FLINK-31092][sql-gateway] Fix OperationManager can not kill the running task by force new 86e12eb3fce [FLINK-31092][table-common] Fix ServiceLoaderUtil keeps loading even though the classloader has been closed The 2 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference. Summary of changes: .../flink/table/gateway/api/utils/ThreadUtils.java | 4 +- .../service/operation/OperationManager.java| 47 +++- .../service/operation/OperationManagerTest.java| 29 .../apache/flink/table/factories/FactoryUtil.java | 52 -- .../flink/table/factories/ServiceLoaderUtil.java | 83 -- .../flink/table/factories/FactoryUtilTest.java | 20 ++ 6 files changed, 128 insertions(+), 107 deletions(-) delete mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/ServiceLoaderUtil.java
[flink] branch master updated: [FLINK-31176][sql-gateway] Correct the description of sql gateway configuration (#21983)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new b252abe179b [FLINK-31176][sql-gateway] Correct the description of sql gateway configuration (#21983) b252abe179b is described below commit b252abe179b5b4458e5306f7bdf8e484ac61d515 Author: wangkang <525262...@qq.com> AuthorDate: Thu Feb 23 12:14:19 2023 +0800 [FLINK-31176][sql-gateway] Correct the description of sql gateway configuration (#21983) Co-authored-by: kandy01.wang --- docs/content.zh/docs/dev/table/sql-gateway/overview.md| 4 ++-- docs/content/docs/dev/table/sql-gateway/overview.md | 4 ++-- .../table/gateway/api/config/SqlGatewayServiceConfigOptions.java | 4 ++-- .../table/gateway/rest/header/session/GetSessionConfigHeaders.java| 2 +- .../gateway/rest/header/session/TriggerSessionHeartbeatHeaders.java | 2 +- 5 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/content.zh/docs/dev/table/sql-gateway/overview.md b/docs/content.zh/docs/dev/table/sql-gateway/overview.md index 2f7f2dd68f2..2bb3ea69cb1 100644 --- a/docs/content.zh/docs/dev/table/sql-gateway/overview.md +++ b/docs/content.zh/docs/dev/table/sql-gateway/overview.md @@ -176,13 +176,13 @@ $ ./sql-gateway -Dkey=value sql-gateway.session.check-interval 1 min Duration -The check interval for idle session timeout, which can be disabled by setting to zero or negative value. +The check interval for idle session timeout, which can be disabled by setting to zero. sql-gateway.session.idle-timeout 10 min Duration -Timeout interval for closing the session when the session hasn't been accessed during the interval. If setting to zero or negative value, the session will not be closed. +Timeout interval for closing the session when the session hasn't been accessed during the interval. If setting to zero, the session will not be closed. sql-gateway.session.max-num diff --git a/docs/content/docs/dev/table/sql-gateway/overview.md b/docs/content/docs/dev/table/sql-gateway/overview.md index 30f9a39de54..99b312cacc6 100644 --- a/docs/content/docs/dev/table/sql-gateway/overview.md +++ b/docs/content/docs/dev/table/sql-gateway/overview.md @@ -176,13 +176,13 @@ $ ./sql-gateway -Dkey=value sql-gateway.session.check-interval 1 min Duration -The check interval for idle session timeout, which can be disabled by setting to zero or negative value. +The check interval for idle session timeout, which can be disabled by setting to zero. sql-gateway.session.idle-timeout 10 min Duration -Timeout interval for closing the session when the session hasn't been accessed during the interval. If setting to zero or negative value, the session will not be closed. +Timeout interval for closing the session when the session hasn't been accessed during the interval. If setting to zero, the session will not be closed. sql-gateway.session.max-num diff --git a/flink-table/flink-sql-gateway-api/src/main/java/org/apache/flink/table/gateway/api/config/SqlGatewayServiceConfigOptions.java b/flink-table/flink-sql-gateway-api/src/main/java/org/apache/flink/table/gateway/api/config/SqlGatewayServiceConfigOptions.java index eadfb85a7f1..f788eac1bdd 100644 --- a/flink-table/flink-sql-gateway-api/src/main/java/org/apache/flink/table/gateway/api/config/SqlGatewayServiceConfigOptions.java +++ b/flink-table/flink-sql-gateway-api/src/main/java/org/apache/flink/table/gateway/api/config/SqlGatewayServiceConfigOptions.java @@ -36,14 +36,14 @@ public class SqlGatewayServiceConfigOptions { .defaultValue(Duration.ofMinutes(10)) .withDescription( "Timeout interval for closing the session when the session hasn't been accessed during the interval. " -+ "If setting to zero or negative value, the session will not be closed."); ++ "If setting to zero, the session will not be closed."); public static final ConfigOption SQL_GATEWAY_SESSION_CHECK_INTERVAL = key("sql-gateway.session.check-interval") .durationType() .defaultValue(Duration.ofMinutes(1)) .withDescription( -"The check interval for idle session timeout, which can be disabled by setting to zero or negative value."); +
[flink] branch master updated: [FLINK-31136][sql-client] Forbid sql client to read executor config in the gateway mode (#21987)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new dec3ba078de [FLINK-31136][sql-client] Forbid sql client to read executor config in the gateway mode (#21987) dec3ba078de is described below commit dec3ba078decbdc212a6ea16ad8728aa7409d9c3 Author: Shengkai <33114724+fsk...@users.noreply.github.com> AuthorDate: Thu Feb 23 11:24:04 2023 +0800 [FLINK-31136][sql-client] Forbid sql client to read executor config in the gateway mode (#21987) --- .../table/client/gateway/DefaultContextUtils.java | 3 +- .../apache/flink/table/client/SqlClientTest.java | 37 +++ .../org/apache/flink/table/gateway/SqlGateway.java | 1 + .../gateway/service/context/DefaultContext.java| 43 +++--- .../service/utils/SqlGatewayServiceExtension.java | 2 +- 5 files changed, 71 insertions(+), 15 deletions(-) diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/DefaultContextUtils.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/DefaultContextUtils.java index b0be702b15c..e7afce568ec 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/DefaultContextUtils.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/DefaultContextUtils.java @@ -54,13 +54,14 @@ public class DefaultContextUtils { } Configuration sessionConfig = options.getPythonConfiguration(); sessionConfig.addAll(ConfigurationUtils.createConfiguration(options.getSessionConfig())); -return DefaultContext.load(sessionConfig, discoverDependencies(jars, libDirs), true); +return DefaultContext.load(sessionConfig, discoverDependencies(jars, libDirs), true, true); } public static DefaultContext buildDefaultContext(CliOptions.GatewayCliOptions options) { return DefaultContext.load( ConfigurationUtils.createConfiguration(options.getSessionConfig()), Collections.emptyList(), +false, false); } // diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/SqlClientTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/SqlClientTest.java index 801b6299c4a..e107b73b1ac 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/SqlClientTest.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/SqlClientTest.java @@ -18,8 +18,11 @@ package org.apache.flink.table.client; +import org.apache.flink.configuration.Configuration; import org.apache.flink.core.testutils.CommonTestUtils; import org.apache.flink.table.client.cli.TerminalUtils; +import org.apache.flink.table.gateway.rest.util.SqlGatewayRestEndpointExtension; +import org.apache.flink.table.gateway.service.utils.SqlGatewayServiceExtension; import org.apache.flink.util.FileUtils; import org.apache.flink.util.Preconditions; @@ -27,7 +30,9 @@ import org.jline.terminal.Size; import org.jline.terminal.Terminal; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Order; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.api.io.TempDir; import java.io.ByteArrayInputStream; @@ -35,6 +40,7 @@ import java.io.ByteArrayOutputStream; import java.io.File; import java.io.IOException; import java.io.OutputStream; +import java.net.InetSocketAddress; import java.net.URL; import java.nio.charset.StandardCharsets; import java.nio.file.Files; @@ -48,6 +54,7 @@ import java.util.List; import java.util.Map; import static org.apache.flink.configuration.ConfigConstants.ENV_FLINK_CONF_DIR; +import static org.apache.flink.configuration.DeploymentOptions.TARGET; import static org.apache.flink.core.testutils.CommonTestUtils.assertThrows; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -57,6 +64,21 @@ class SqlClientTest { @TempDir private Path tempFolder; +@RegisterExtension +@Order(1) +public static final SqlGatewayServiceExtension SQL_GATEWAY_SERVICE_EXTENSION = +new SqlGatewayServiceExtension( +() -> { +Configuration configuration = new Configuration(); +configuration.set(TARGET, "yarn-session"); +return configuration; +}); + +@RegisterExtension +@Order(2) +private static final SqlGatewayRe
[flink] branch release-1.17 updated: [FLINK-31175][doc] Fix link doesn't work in hive_read_write doc (#21988)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch release-1.17 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.17 by this push: new fb901d19ebc [FLINK-31175][doc] Fix link doesn't work in hive_read_write doc (#21988) fb901d19ebc is described below commit fb901d19ebc392bbb29336c752e6c18776ba9036 Author: Shengkai <33114724+fsk...@users.noreply.github.com> AuthorDate: Wed Feb 22 17:20:14 2023 +0800 [FLINK-31175][doc] Fix link doesn't work in hive_read_write doc (#21988) --- docs/content/docs/connectors/table/hive/hive_read_write.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/content/docs/connectors/table/hive/hive_read_write.md b/docs/content/docs/connectors/table/hive/hive_read_write.md index f02270e8056..c5bc91edb11 100644 --- a/docs/content/docs/connectors/table/hive/hive_read_write.md +++ b/docs/content/docs/connectors/table/hive/hive_read_write.md @@ -597,7 +597,7 @@ configured threshold, then Flink will try to compact these files to files with a yes 16MB MemorySize -The threshold for file compaction. If the average size of the files is less than this value, FLink will then compact these files. the default value is 16MB. +The threshold for file compaction. If the average size of the files is less than this value, Flink will then compact these files. the default value is 16MB. compaction.file-size @@ -605,7 +605,7 @@ configured threshold, then Flink will try to compact these files to files with a yes (none) MemorySize -The compaction target file size, the default value is the }}#sink.rolling-policy.file-size">rolling file size. +The compaction target file size, the default value is the }}#sink-rolling-policy-file-size">rolling file size. compaction.parallelism
[flink] branch master updated (b3e14928e81 -> 3a64195c495)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from b3e14928e81 [FLINK-31041][runtime] Fix multiple restoreState when GlobalFailure occurs in a short period. add 3a64195c495 [FLINK-31175][doc] Fix link doesn't work in hive_read_write doc (#21988) No new revisions were added by this update. Summary of changes: docs/content/docs/connectors/table/hive/hive_read_write.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-)
[flink] branch release-1.17 updated: [FLINK-31136][sql-client] Forbid sql client to read executor config in the gateway mode
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch release-1.17 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.17 by this push: new db15e1e2014 [FLINK-31136][sql-client] Forbid sql client to read executor config in the gateway mode db15e1e2014 is described below commit db15e1e2014f02cbeb58d8a4fee1befdbd5a3ac8 Author: Shengkai <33114724+fsk...@users.noreply.github.com> AuthorDate: Wed Feb 22 10:46:28 2023 +0800 [FLINK-31136][sql-client] Forbid sql client to read executor config in the gateway mode This closes #21974 --- .../table/client/gateway/DefaultContextUtils.java | 4 +- .../apache/flink/table/client/SqlClientTest.java | 37 +++ .../org/apache/flink/table/gateway/SqlGateway.java | 1 + .../gateway/service/context/DefaultContext.java| 43 +++--- .../service/utils/SqlGatewayServiceExtension.java | 2 +- 5 files changed, 71 insertions(+), 16 deletions(-) diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/DefaultContextUtils.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/DefaultContextUtils.java index cda02c794c9..c4f82d7272b 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/DefaultContextUtils.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/DefaultContextUtils.java @@ -52,11 +52,11 @@ public class DefaultContextUtils { libDirs = Collections.emptyList(); } return DefaultContext.load( -options.getPythonConfiguration(), discoverDependencies(jars, libDirs), true); +options.getPythonConfiguration(), discoverDependencies(jars, libDirs), true, true); } public static DefaultContext buildDefaultContext(CliOptions.GatewayCliOptions options) { -return DefaultContext.load(new Configuration(), Collections.emptyList(), false); +return DefaultContext.load(new Configuration(), Collections.emptyList(), false, false); } // diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/SqlClientTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/SqlClientTest.java index a0e70edd1f6..e93b6edb2e3 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/SqlClientTest.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/SqlClientTest.java @@ -18,8 +18,11 @@ package org.apache.flink.table.client; +import org.apache.flink.configuration.Configuration; import org.apache.flink.core.testutils.CommonTestUtils; import org.apache.flink.table.client.cli.TerminalUtils; +import org.apache.flink.table.gateway.rest.util.SqlGatewayRestEndpointExtension; +import org.apache.flink.table.gateway.service.utils.SqlGatewayServiceExtension; import org.apache.flink.util.FileUtils; import org.apache.flink.util.Preconditions; @@ -27,7 +30,9 @@ import org.jline.terminal.Size; import org.jline.terminal.Terminal; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Order; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.api.io.TempDir; import java.io.ByteArrayInputStream; @@ -35,6 +40,7 @@ import java.io.ByteArrayOutputStream; import java.io.File; import java.io.IOException; import java.io.OutputStream; +import java.net.InetSocketAddress; import java.net.URL; import java.nio.charset.StandardCharsets; import java.nio.file.Files; @@ -48,6 +54,7 @@ import java.util.List; import java.util.Map; import static org.apache.flink.configuration.ConfigConstants.ENV_FLINK_CONF_DIR; +import static org.apache.flink.configuration.DeploymentOptions.TARGET; import static org.apache.flink.core.testutils.CommonTestUtils.assertThrows; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -57,6 +64,21 @@ class SqlClientTest { @TempDir private Path tempFolder; +@RegisterExtension +@Order(1) +public static final SqlGatewayServiceExtension SQL_GATEWAY_SERVICE_EXTENSION = +new SqlGatewayServiceExtension( +() -> { +Configuration configuration = new Configuration(); +configuration.set(TARGET, "yarn-session"); +return configuration; +}); + +@RegisterExtension +@Order(2) +private static final SqlGatewayRestEndpointExtension SQL_GATEWAY_REST_ENDPOINT_EXTENSION = +new SqlGatewayRe
[flink] branch master updated: [FLINK-31091][sql-gateway] Add Ser/de for Interval types (#21945)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new 14adc1679fb [FLINK-31091][sql-gateway] Add Ser/de for Interval types (#21945) 14adc1679fb is described below commit 14adc1679fb3d025a2808af91f23f14e7c6f6e24 Author: Sergey Nuyanzin AuthorDate: Wed Feb 22 03:30:19 2023 +0100 [FLINK-31091][sql-gateway] Add Ser/de for Interval types (#21945) --- .../src/test/resources/sql/select.q| 9 .../src/test/resources/sql/select_batch.q | 9 .../rest/serde/LogicalTypeJsonDeserializer.java| 29 +++ .../rest/serde/LogicalTypeJsonSerializer.java | 33 +++- .../rest/serde/LogicalTypeJsonSerDeTest.java | 59 +++--- .../rest/serde/ResultInfoJsonSerDeTest.java| 38 +- .../src/test/resources/result_info_json_format.txt | 2 +- .../resources/result_info_plain_text_format.txt| 2 +- 8 files changed, 158 insertions(+), 23 deletions(-) diff --git a/flink-table/flink-sql-client/src/test/resources/sql/select.q b/flink-table/flink-sql-client/src/test/resources/sql/select.q index a562db288e1..6029c902f6b 100644 --- a/flink-table/flink-sql-client/src/test/resources/sql/select.q +++ b/flink-table/flink-sql-client/src/test/resources/sql/select.q @@ -327,3 +327,12 @@ DROP TEMPORARY VIEW testUserData; SET 'sql-client.display.max-column-width' = '30'; [INFO] Execute statement succeed. !info + +SELECT INTERVAL '1' DAY as dayInterval, INTERVAL '1' YEAR as yearInterval; ++-+--+ +| dayInterval | yearInterval | ++-+--+ +| +1 00:00:00.000 |+1-00 | ++-+--+ +1 row in set +!ok diff --git a/flink-table/flink-sql-client/src/test/resources/sql/select_batch.q b/flink-table/flink-sql-client/src/test/resources/sql/select_batch.q index 955ff8ccca3..8a5983ee765 100644 --- a/flink-table/flink-sql-client/src/test/resources/sql/select_batch.q +++ b/flink-table/flink-sql-client/src/test/resources/sql/select_batch.q @@ -127,3 +127,12 @@ DROP TEMPORARY VIEW testUserData; SET 'sql-client.display.max-column-width' = '30'; [INFO] Execute statement succeed. !info + +SELECT INTERVAL '1' DAY as dayInterval, INTERVAL '1' YEAR as yearInterval; ++-+--+ +| dayInterval | yearInterval | ++-+--+ +| +1 00:00:00.000 |+1-00 | ++-+--+ +1 row in set +!ok diff --git a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/serde/LogicalTypeJsonDeserializer.java b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/serde/LogicalTypeJsonDeserializer.java index 63d2bbd1b4b..c1da541dc39 100644 --- a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/serde/LogicalTypeJsonDeserializer.java +++ b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/serde/LogicalTypeJsonDeserializer.java @@ -27,6 +27,7 @@ import org.apache.flink.table.types.logical.BinaryType; import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.CharType; import org.apache.flink.table.types.logical.DateType; +import org.apache.flink.table.types.logical.DayTimeIntervalType; import org.apache.flink.table.types.logical.DecimalType; import org.apache.flink.table.types.logical.DoubleType; import org.apache.flink.table.types.logical.FloatType; @@ -46,6 +47,7 @@ import org.apache.flink.table.types.logical.TimestampType; import org.apache.flink.table.types.logical.TinyIntType; import org.apache.flink.table.types.logical.VarBinaryType; import org.apache.flink.table.types.logical.VarCharType; +import org.apache.flink.table.types.logical.YearMonthIntervalType; import org.apache.flink.table.types.logical.ZonedTimestampType; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser; @@ -64,10 +66,12 @@ import static org.apache.flink.table.gateway.rest.serde.LogicalTypeJsonSerialize import static org.apache.flink.table.gateway.rest.serde.LogicalTypeJsonSerializer.FIELD_NAME_FIELD_NAME; import static org.apache.flink.table.gateway.rest.serde.LogicalTypeJsonSerializer.FIELD_NAME_FIELD_TYPE; import static org.apache.flink.table.gateway.rest.serde.LogicalTypeJsonSerializer.FIELD_NAME_FILED_DESCRIPTION; +import static org.apache.flink.table.gateway.rest.serde.LogicalTypeJsonSerializer.FIELD_NAME_FRACTIONAL_PRECISION; import static org.apache.flink.table.gateway.rest.serde.LogicalTypeJsonSerializer.FIELD_NAME_KEY_TYPE; import static org.apache.flink.table.gateway.rest.serde.LogicalTypeJsonSerializer.FIELD_NAME_LENGTH; import static org.apache.flink
[flink] branch master updated (da00b9c7de7 -> 0508d82ae93)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from da00b9c7de7 [FLINK-30968][sql-client] Sql client supports dynamic config to open session (#21901) add 0508d82ae93 [FLINK-31137][hive] Fix wrong ResultKind in DescribeTable and ShowCreateTable results (#21980) No new revisions were added by this update. Summary of changes: .../planner/delegation/hive/HiveOperationExecutor.java| 4 ++-- .../apache/flink/connectors/hive/HiveDialectITCase.java | 15 +++ 2 files changed, 13 insertions(+), 6 deletions(-)
[flink] branch master updated: [FLINK-30968][sql-client] Sql client supports dynamic config to open session (#21901)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new da00b9c7de7 [FLINK-30968][sql-client] Sql client supports dynamic config to open session (#21901) da00b9c7de7 is described below commit da00b9c7de7a496d2fe42d0f94836fd7451676aa Author: Shammon FY AuthorDate: Tue Feb 21 19:59:17 2023 +0800 [FLINK-30968][sql-client] Sql client supports dynamic config to open session (#21901) --- docs/content.zh/docs/dev/table/sqlClient.md| 4 +++ docs/content/docs/dev/table/sqlClient.md | 4 +++ .../apache/flink/table/client/cli/CliOptions.java | 34 ++ .../flink/table/client/cli/CliOptionsParser.java | 16 -- .../table/client/gateway/DefaultContextUtils.java | 11 +-- .../apache/flink/table/client/SqlClientTest.java | 7 + .../src/test/resources/cli/all-mode-help.out | 4 +++ .../src/test/resources/cli/embedded-mode-help.out | 2 ++ .../src/test/resources/cli/gateway-mode-help.out | 2 ++ 9 files changed, 74 insertions(+), 10 deletions(-) diff --git a/docs/content.zh/docs/dev/table/sqlClient.md b/docs/content.zh/docs/dev/table/sqlClient.md index 130a08be6a8..4a5961c7a99 100644 --- a/docs/content.zh/docs/dev/table/sqlClient.md +++ b/docs/content.zh/docs/dev/table/sqlClient.md @@ -223,6 +223,8 @@ Mode "embedded" (default) submits Flink jobs from the local machine. Syntax: [embedded] [OPTIONS] "embedded" mode options: + -D The dynamic config key=val for a +session. -f,--file
[flink] branch release-1.17 updated: [FLINK-31137][hive] Fix wrong ResultKind in DescribeTable and ShowCreateTable results (#21976)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch release-1.17 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.17 by this push: new 3ae7b1f3d85 [FLINK-31137][hive] Fix wrong ResultKind in DescribeTable and ShowCreateTable results (#21976) 3ae7b1f3d85 is described below commit 3ae7b1f3d85db71f0950f1166d60be76720b49f5 Author: Shengkai <33114724+fsk...@users.noreply.github.com> AuthorDate: Tue Feb 21 14:25:35 2023 +0800 [FLINK-31137][hive] Fix wrong ResultKind in DescribeTable and ShowCreateTable results (#21976) --- .../planner/delegation/hive/HiveOperationExecutor.java| 4 ++-- .../apache/flink/connectors/hive/HiveDialectITCase.java | 15 +++ 2 files changed, 13 insertions(+), 6 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/HiveOperationExecutor.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/HiveOperationExecutor.java index 221621888f5..ee23c36fa90 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/HiveOperationExecutor.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/HiveOperationExecutor.java @@ -267,7 +267,7 @@ public class HiveOperationExecutor implements ExtendedOperationExecutor { String showCreateTableString = HiveShowTableUtils.showCreateTable(tablePath, tbl); TableResultInternal resultInternal = TableResultImpl.builder() -.resultKind(ResultKind.SUCCESS) +.resultKind(ResultKind.SUCCESS_WITH_CONTENT) .schema(ResolvedSchema.of(Column.physical("result", DataTypes.STRING( .data(Collections.singletonList(Row.of(showCreateTableString))) .build(); @@ -325,7 +325,7 @@ public class HiveOperationExecutor implements ExtendedOperationExecutor { } TableResultInternal tableResultInternal = TableResultImpl.builder() -.resultKind(ResultKind.SUCCESS) +.resultKind(ResultKind.SUCCESS_WITH_CONTENT) .schema( ResolvedSchema.physical( new String[] {"col_name", "data_type", "comment"}, 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 1b10e6c93a7..d93c10ea39a 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 @@ -20,9 +20,11 @@ package org.apache.flink.connectors.hive; import org.apache.flink.sql.parser.hive.ddl.SqlCreateHiveTable; import org.apache.flink.table.HiveVersionTestUtil; +import org.apache.flink.table.api.ResultKind; import org.apache.flink.table.api.SqlDialect; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableResult; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.api.internal.TableEnvironmentImpl; @@ -1211,10 +1213,11 @@ public class HiveDialectITCase { "default.t1")); // show hive table +TableResult showCreateTableT2 = tableEnv.executeSql("show create table t2"); + assertThat(showCreateTableT2.getResultKind()).isEqualTo(ResultKind.SUCCESS_WITH_CONTENT); String actualResult = (String) -CollectionUtil.iteratorToList( -tableEnv.executeSql("show create table t2").collect()) + CollectionUtil.iteratorToList(showCreateTableT2.collect()) .get(0) .getField(0); Table table = hiveCatalog.getHiveTable(new ObjectPath("default", "t2")); @@ -1264,12 +1267,16 @@ public class HiveDialectITCase { "create table t3(a decimal(10, 2), b double, c float) partitioned by (d date)"); // desc non-hive table -List result = CollectionUtil.iteratorToList(tableEnv.executeSql("desc t1").collect()); +TableResult descT1 = tableEnv.executeSql("desc t1"); + assertThat(d
[flink] branch release-1.17 updated: [FLINK-28658][docs] Add docs for job statements
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch release-1.17 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.17 by this push: new e68bac9cce9 [FLINK-28658][docs] Add docs for job statements e68bac9cce9 is described below commit e68bac9cce9a1e3a68c49847b4b56bb47603f257 Author: Paul Lin AuthorDate: Sat Feb 11 17:17:08 2023 +0800 [FLINK-28658][docs] Add docs for job statements This closes #21912 --- docs/content.zh/docs/dev/table/sql/jar.md | 2 +- docs/content.zh/docs/dev/table/sql/job.md | 96 docs/content.zh/docs/dev/table/sql/show.md | 12 +- docs/content.zh/docs/dev/table/sqlClient.md| 161 - .../content/docs/dev/table/sql-gateway/overview.md | 2 +- docs/content/docs/dev/table/sql/jar.md | 4 +- docs/content/docs/dev/table/sql/job.md | 97 + docs/content/docs/dev/table/sql/show.md| 13 +- docs/content/docs/dev/table/sqlClient.md | 49 ++- 9 files changed, 416 insertions(+), 20 deletions(-) diff --git a/docs/content.zh/docs/dev/table/sql/jar.md b/docs/content.zh/docs/dev/table/sql/jar.md index c51698e20d5..f6857ee753e 100644 --- a/docs/content.zh/docs/dev/table/sql/jar.md +++ b/docs/content.zh/docs/dev/table/sql/jar.md @@ -1,5 +1,5 @@ --- -title: "JAR Statements" +title: "JAR 语句" weight: 16 type: docs aliases: diff --git a/docs/content.zh/docs/dev/table/sql/job.md b/docs/content.zh/docs/dev/table/sql/job.md new file mode 100644 index 000..e3ce4783ebb --- /dev/null +++ b/docs/content.zh/docs/dev/table/sql/job.md @@ -0,0 +1,96 @@ +--- +title: "JOB 语句" +weight: 16 +type: docs +aliases: +- /dev/table/sql/job.html +--- + + +# JOB Statements + +Job 语句用于管理作业的生命周期。 + +目前 Flink SQL 支持以下 JOB 语句: +- SHOW JOBS +- STOP JOB + +## 执行 JOB 语句 + +{{< tabs "show jobs statement" >}} +{{< tab "SQL CLI" >}} + +以下示例展示如何在 [SQL CLI]({{< ref "docs/dev/table/sqlClient" >}}) 中执行 JOB 语句. + +{{< /tab >}} +{{< /tabs >}} + +{{< tabs "show jobs" >}} +{{< tab "SQL CLI" >}} +```sql +Flink SQL> SHOW JOBS; ++--+--+-+-+ +| job id | job name | status | start time | ++--+--+-+-+ +| 228d70913eab60dda85c5e7f78b5782c |myjob | RUNNING | 2023-02-11T05:03:51.523 | ++--+--+-+-+ + +Flink SQL> SET 'state.savepoints.dir'='file:/tmp/'; +[INFO] Execute statement succeed. + +Flink SQL> STOP JOB '228d70913eab60dda85c5e7f78b5782c' WITH SAVEPOINT; ++-+ +| savepoint path | ++-+ +| file:/tmp/savepoint-3addd4-0b224d9311e6 | ++-+ +``` +{{< /tab >}} +{{< /tabs >}} + +## SHOW JOBS + +```sql +SHOW JOBS +``` + +展示 Flink 集群上的作业。 + +Attention SHOW JOBS 语句仅适用于 [SQL CLI]({{< ref "docs/dev/table/sqlClient" >}}) 或者 [SQL Gateway]({{< ref "docs/dev/table/sql-gateway/overview" >}}). + +## STOP JOB + +```sql +STOP JOB '' [WITH SAVEPOINT] [WITH DRAIN] +``` + +停止指定作业。 + +**WITH SAVEPOINT** +在作业停止之前执行 Savepoin。 Savepoint 的路径可以通过集群配置的 +[state.savepoints.dir]({{< ref "docs/deployment/config" >}}#state-savepoints-dir) 指定, +或者通过 `SET` 语句指定(后者有更高优先级)。 + +**WITH DRAIN** +在触发 savepoint 之前将 Watermark 提升至最大。该操作会可能会触发窗口的计算。请您注意该操作可能导致您之后从该创建的 savepoint 恢复的作业结果不正确。 + +Attention STOP JOB 语句仅适用于 [SQL CLI]({{< ref "docs/dev/table/sqlClient" >}}) 或者 [SQL Gateway]({{< ref "docs/dev/table/sql-gateway/overview" >}}). + +{{< top >}} diff --git a/docs/content.zh/docs/dev/table/sql/show.md b/docs/content.zh/docs/dev/table/sql/show.md index 76210678b9d..3bbc5b3d7c7 100644 --- a/docs/content.zh/docs/dev/table/sql/show.md +++ b/docs/content.zh/docs/dev/table/sql/show.md @@ -746,6 +746,16 @@ SHOW JARS 展示所有通过 [`ADD JAR`]({{< ref "docs/dev/table/sql/jar" >}}#add-jar) 语句加入到 session classloader 中的 jar。 -Attention 当前 SHOW JARS 命令只能在 [SQL CLI]({{< ref "docs/dev/table/sqlClient" >}}) 中使用。 +Attention 当前 SHOW JARS 命令只能在 [SQL CLI]({{< ref "docs/dev/table/sqlClient" >}}) 或者 [SQL Gateway]({{< ref "docs/dev/table/sql-gateway/overview" >}}) 中使用. + +## SHOW JOBS + +```sql +SHOW JOBS +``` + +展示集群中所有作业。 + +Attention 当前 SHOW JOBS 命令只能在 [SQL CLI]({{< ref "docs/dev/table/sqlClient" >}}) 或者 [SQL Gateway]({{< ref "docs/dev/table/sql-gateway/overview" &g
[flink] branch master updated: [FLINK-28658][docs] Add docs for job statements
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new 00e1ef38f98 [FLINK-28658][docs] Add docs for job statements 00e1ef38f98 is described below commit 00e1ef38f98d176b2ed00aed3176b66d9b75ec9c Author: Paul Lin AuthorDate: Sat Feb 11 17:17:08 2023 +0800 [FLINK-28658][docs] Add docs for job statements This closes #21912 --- docs/content.zh/docs/dev/table/sql/jar.md | 2 +- docs/content.zh/docs/dev/table/sql/job.md | 96 docs/content.zh/docs/dev/table/sql/show.md | 12 +- docs/content.zh/docs/dev/table/sqlClient.md| 161 - .../content/docs/dev/table/sql-gateway/overview.md | 2 +- docs/content/docs/dev/table/sql/jar.md | 4 +- docs/content/docs/dev/table/sql/job.md | 97 + docs/content/docs/dev/table/sql/show.md| 13 +- docs/content/docs/dev/table/sqlClient.md | 49 ++- 9 files changed, 416 insertions(+), 20 deletions(-) diff --git a/docs/content.zh/docs/dev/table/sql/jar.md b/docs/content.zh/docs/dev/table/sql/jar.md index c51698e20d5..f6857ee753e 100644 --- a/docs/content.zh/docs/dev/table/sql/jar.md +++ b/docs/content.zh/docs/dev/table/sql/jar.md @@ -1,5 +1,5 @@ --- -title: "JAR Statements" +title: "JAR 语句" weight: 16 type: docs aliases: diff --git a/docs/content.zh/docs/dev/table/sql/job.md b/docs/content.zh/docs/dev/table/sql/job.md new file mode 100644 index 000..e3ce4783ebb --- /dev/null +++ b/docs/content.zh/docs/dev/table/sql/job.md @@ -0,0 +1,96 @@ +--- +title: "JOB 语句" +weight: 16 +type: docs +aliases: +- /dev/table/sql/job.html +--- + + +# JOB Statements + +Job 语句用于管理作业的生命周期。 + +目前 Flink SQL 支持以下 JOB 语句: +- SHOW JOBS +- STOP JOB + +## 执行 JOB 语句 + +{{< tabs "show jobs statement" >}} +{{< tab "SQL CLI" >}} + +以下示例展示如何在 [SQL CLI]({{< ref "docs/dev/table/sqlClient" >}}) 中执行 JOB 语句. + +{{< /tab >}} +{{< /tabs >}} + +{{< tabs "show jobs" >}} +{{< tab "SQL CLI" >}} +```sql +Flink SQL> SHOW JOBS; ++--+--+-+-+ +| job id | job name | status | start time | ++--+--+-+-+ +| 228d70913eab60dda85c5e7f78b5782c |myjob | RUNNING | 2023-02-11T05:03:51.523 | ++--+--+-+-+ + +Flink SQL> SET 'state.savepoints.dir'='file:/tmp/'; +[INFO] Execute statement succeed. + +Flink SQL> STOP JOB '228d70913eab60dda85c5e7f78b5782c' WITH SAVEPOINT; ++-+ +| savepoint path | ++-+ +| file:/tmp/savepoint-3addd4-0b224d9311e6 | ++-+ +``` +{{< /tab >}} +{{< /tabs >}} + +## SHOW JOBS + +```sql +SHOW JOBS +``` + +展示 Flink 集群上的作业。 + +Attention SHOW JOBS 语句仅适用于 [SQL CLI]({{< ref "docs/dev/table/sqlClient" >}}) 或者 [SQL Gateway]({{< ref "docs/dev/table/sql-gateway/overview" >}}). + +## STOP JOB + +```sql +STOP JOB '' [WITH SAVEPOINT] [WITH DRAIN] +``` + +停止指定作业。 + +**WITH SAVEPOINT** +在作业停止之前执行 Savepoin。 Savepoint 的路径可以通过集群配置的 +[state.savepoints.dir]({{< ref "docs/deployment/config" >}}#state-savepoints-dir) 指定, +或者通过 `SET` 语句指定(后者有更高优先级)。 + +**WITH DRAIN** +在触发 savepoint 之前将 Watermark 提升至最大。该操作会可能会触发窗口的计算。请您注意该操作可能导致您之后从该创建的 savepoint 恢复的作业结果不正确。 + +Attention STOP JOB 语句仅适用于 [SQL CLI]({{< ref "docs/dev/table/sqlClient" >}}) 或者 [SQL Gateway]({{< ref "docs/dev/table/sql-gateway/overview" >}}). + +{{< top >}} diff --git a/docs/content.zh/docs/dev/table/sql/show.md b/docs/content.zh/docs/dev/table/sql/show.md index 76210678b9d..3bbc5b3d7c7 100644 --- a/docs/content.zh/docs/dev/table/sql/show.md +++ b/docs/content.zh/docs/dev/table/sql/show.md @@ -746,6 +746,16 @@ SHOW JARS 展示所有通过 [`ADD JAR`]({{< ref "docs/dev/table/sql/jar" >}}#add-jar) 语句加入到 session classloader 中的 jar。 -Attention 当前 SHOW JARS 命令只能在 [SQL CLI]({{< ref "docs/dev/table/sqlClient" >}}) 中使用。 +Attention 当前 SHOW JARS 命令只能在 [SQL CLI]({{< ref "docs/dev/table/sqlClient" >}}) 或者 [SQL Gateway]({{< ref "docs/dev/table/sql-gateway/overview" >}}) 中使用. + +## SHOW JOBS + +```sql +SHOW JOBS +``` + +展示集群中所有作业。 + +Attention 当前 SHOW JOBS 命令只能在 [SQL CLI]({{< ref "docs/dev/table/sqlClient" >}}) 或者 [SQL Gateway]({{< ref "docs/dev/table/sql-gateway/overview" >}}) 中使用.
[flink] branch release-1.16 updated (700f8839126 -> 0994832be8a)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git from 700f8839126 [FLINK-28440][state/changelog] record reference count of StreamStateHandle in TaskChangelogRegistry add 0994832be8a [FLINK-30640][sql-client] Fix unstable ctas test in CliClientITCase (#21896) No new revisions were added by this update. Summary of changes: flink-table/flink-sql-client/src/test/resources/sql/set.q | 11 +-- 1 file changed, 9 insertions(+), 2 deletions(-)
[flink] branch release-1.17 updated: [FLINK-30940][sql-client] Redirect the interrupted exception to the log (#21891)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch release-1.17 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.17 by this push: new 72cf4eaa423 [FLINK-30940][sql-client] Redirect the interrupted exception to the log (#21891) 72cf4eaa423 is described below commit 72cf4eaa42379ad38fd6bc7cb52330af070cfe63 Author: Shengkai <33114724+fsk...@users.noreply.github.com> AuthorDate: Thu Feb 9 19:53:15 2023 +0800 [FLINK-30940][sql-client] Redirect the interrupted exception to the log (#21891) --- .../org/apache/flink/table/client/gateway/ExecutorImplITCase.java | 8 +++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/ExecutorImplITCase.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/ExecutorImplITCase.java index 770c4ea6f61..88447264246 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/ExecutorImplITCase.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/ExecutorImplITCase.java @@ -58,6 +58,7 @@ import org.apache.flink.table.gateway.rest.util.SqlGatewayRestEndpointExtension; import org.apache.flink.table.gateway.rest.util.TestingSqlGatewayRestEndpoint; import org.apache.flink.table.gateway.service.context.DefaultContext; import org.apache.flink.table.gateway.service.session.SessionManagerImpl; +import org.apache.flink.table.gateway.service.utils.IgnoreExceptionHandler; import org.apache.flink.table.gateway.service.utils.SqlGatewayServiceExtension; import org.apache.flink.table.utils.UserDefinedFunctions; import org.apache.flink.table.utils.print.RowDataToStringConverter; @@ -68,6 +69,7 @@ import org.apache.flink.test.util.TestUtils; import org.apache.flink.util.CollectionUtil; import org.apache.flink.util.StringUtils; import org.apache.flink.util.UserClassLoaderJarTestUtils; +import org.apache.flink.util.concurrent.ExecutorThreadFactory; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Order; @@ -96,6 +98,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ThreadFactory; import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -164,6 +167,9 @@ class ExecutorImplITCase { // a generated UDF jar used for testing classloading of dependencies private static URL udfDependency; +private final ThreadFactory threadFactory = +new ExecutorThreadFactory("Executor Test Pool", IgnoreExceptionHandler.INSTANCE); + @BeforeAll static void setup(@InjectClusterClient RestClusterClient injectedClusterClient) throws Exception { @@ -604,7 +610,7 @@ class ExecutorImplITCase { private void testInterrupting(Consumer task) throws Exception { try (Executor executor = createTestServiceExecutor()) { -Thread t = new Thread(() -> task.accept(executor), "worker"); +Thread t = threadFactory.newThread(() -> task.accept(executor)); t.start(); TestSqlGatewayService service =
[flink] branch master updated (df23acfc0f1 -> 8ff06420312)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from df23acfc0f1 [FLINK-30944] Fix thread leak in ExecutionGraphPartitionReleaseTest. add 8ff06420312 [FLINK-30940][sql-client] Redirect the interrupted exception to the log (#21891) No new revisions were added by this update. Summary of changes: .../org/apache/flink/table/client/gateway/ExecutorImplITCase.java | 8 +++- 1 file changed, 7 insertions(+), 1 deletion(-)
[flink] branch release-1.17 updated: [FLINK-30640][sql-client] Fix unstable ctas test in CliClientITCase (#21892)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch release-1.17 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.17 by this push: new 7cc2b4f35ea [FLINK-30640][sql-client] Fix unstable ctas test in CliClientITCase (#21892) 7cc2b4f35ea is described below commit 7cc2b4f35ea54c260365cdbc3e1ba92a06c5b112 Author: Ron AuthorDate: Thu Feb 9 10:32:50 2023 +0800 [FLINK-30640][sql-client] Fix unstable ctas test in CliClientITCase (#21892) --- flink-table/flink-sql-client/src/test/resources/sql/set.q | 11 --- 1 file changed, 8 insertions(+), 3 deletions(-) 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 d375dc0845c..80e7ead6115 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 @@ -74,12 +74,17 @@ CREATE TABLE hive_table ( [INFO] Execute statement succeed. !info +SET table.dml-sync = true; +[INFO] Execute statement succeed. +!info + # test "ctas" in Hive Dialect CREATE TABLE foo as select 1; -[INFO] Submitting SQL update statement to the cluster... -[INFO] SQL update statement has been successfully submitted to the cluster: -Job ID: +[INFO] Complete execution of the SQL update statement. +!info +RESET table.dml-sync; +[INFO] Execute statement succeed. !info SELECT * from foo;
[flink] branch master updated (e33034f8aa3 -> 550697af3df)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from e33034f8aa3 [FLINK-30958][rest][docs] Fix REST API doc generation failure caused by JobClientHeartbeatHeaders add 550697af3df [FLINK-30640][sql-client] Fix unstable ctas test in CliClientITCase (#21892) No new revisions were added by this update. Summary of changes: flink-table/flink-sql-client/src/test/resources/sql/set.q | 11 --- 1 file changed, 8 insertions(+), 3 deletions(-)
[flink] 02/02: [FLINK-30925][docs] Add docs about sql-client remote mode
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch release-1.17 in repository https://gitbox.apache.org/repos/asf/flink.git commit 78e2c9a13043d926bb1c4efb62f45cf4ef38561f Author: Shengkai <1059623...@qq.com> AuthorDate: Mon Feb 6 19:56:14 2023 +0800 [FLINK-30925][docs] Add docs about sql-client remote mode This closes #21877 --- docs/content.zh/docs/dev/table/sqlClient.md | 297 --- docs/content/docs/dev/table/sqlClient.md| 298 +--- 2 files changed, 363 insertions(+), 232 deletions(-) diff --git a/docs/content.zh/docs/dev/table/sqlClient.md b/docs/content.zh/docs/dev/table/sqlClient.md index d5e93cc585a..9d89ef29233 100644 --- a/docs/content.zh/docs/dev/table/sqlClient.md +++ b/docs/content.zh/docs/dev/table/sqlClient.md @@ -47,7 +47,7 @@ SQL 客户端捆绑在常规 Flink 发行版中,因此可以直接运行。它 ### 启动 SQL 客户端命令行界面 -SQL Client 脚本也位于 Flink 的 bin 目录中。[将来](#局限与未来),用户可以通过启动嵌入式 standalone 进程或通过连接到远程 SQL 客户端网关来启动 SQL 客户端命令行界面。目前仅支持 `embedded`,模式默认值`embedded`。可以通过以下方式启动 CLI: +SQL Client 脚本也位于 Flink 的 bin 目录中。用户可以通过启动嵌入式 standalone 进程或通过连接到远程 [SQL Gateway]({{< ref "docs/dev/table/sql-gateway/overview" >}}) 来启动 SQL 客户端命令行界面。SQL 客户端默认使用 `embedded` 模式,你可以通过以下方式启动 CLI: ```bash ./bin/sql-client.sh @@ -59,6 +59,16 @@ SQL Client 脚本也位于 Flink 的 bin 目录中。[将来](#局限与未来) ./bin/sql-client.sh embedded ``` +若想使用 gateway 模式,你可以通过以下命令启动 SQL 客户端: + +```bash +./bin/sql-client.sh gateway --endpoint +``` + +Note SQL 客户端目前只支持和 REST API 版本大于 v1 的 [REST Endpoint]({{< ref "docs/dev/table/sql-gateway/rest" >}}#rest-api) 通信。 + +参阅 [SQL Client startup options](#sql-client-startup-options) 了解更多启动命令。 + ### 执行 SQL 查询 命令行界面启动后,你可以使用 `HELP` 命令列出所有可用的 SQL 语句。输入第一条 SQL 查询语句并按 `Enter` 键执行,可以验证你的设置及集群连接是否正确: @@ -199,118 +209,180 @@ Configuration The SQL Client can be started with the following optional CLI commands. They are discussed in detail in the subsequent paragraphs. ```text -./bin/sql-client.sh --help +./sql-client [MODE] [OPTIONS] + +The following options are available: Mode "embedded" (default) submits Flink jobs from the local machine. Syntax: [embedded] [OPTIONS] "embedded" mode options: - -f,--file
[flink] branch release-1.17 updated (d2e927dc68d -> 78e2c9a1304)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch release-1.17 in repository https://gitbox.apache.org/repos/asf/flink.git from d2e927dc68d [FLINK-22320][docs][table] Add documentation for new introduced ALTER TABLE statements new ace89597edb [FLINK-30925][docs] Update sql-gateway open api doc new 78e2c9a1304 [FLINK-30925][docs] Add docs about sql-client remote mode The 2 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference. Summary of changes: docs/content.zh/docs/dev/table/sql-gateway/rest.md | 18 +- docs/content.zh/docs/dev/table/sqlClient.md| 297 docs/content/docs/dev/table/sql-gateway/rest.md| 18 +- docs/content/docs/dev/table/sqlClient.md | 298 + .../shortcodes/generated/rest_v1_dispatcher.html | 66 - .../shortcodes/generated/rest_v1_sql_gateway.html | 14 +- ...1_sql_gateway.html => rest_v2_sql_gateway.html} | 142 +- docs/static/generated/rest_v1_sql_gateway.yml | 48 +++- ..._v1_sql_gateway.yml => rest_v2_sql_gateway.yml} | 111 +++- 9 files changed, 728 insertions(+), 284 deletions(-) copy docs/layouts/shortcodes/generated/{rest_v1_sql_gateway.html => rest_v2_sql_gateway.html} (81%) copy docs/static/generated/{rest_v1_sql_gateway.yml => rest_v2_sql_gateway.yml} (81%)
[flink] 01/02: [FLINK-30925][docs] Update sql-gateway open api doc
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch release-1.17 in repository https://gitbox.apache.org/repos/asf/flink.git commit ace89597edb5074a7455d6d69b0d087bdfb8704e Author: Shengkai <1059623...@qq.com> AuthorDate: Mon Feb 6 19:49:22 2023 +0800 [FLINK-30925][docs] Update sql-gateway open api doc --- docs/content.zh/docs/dev/table/sql-gateway/rest.md | 18 ++- docs/content/docs/dev/table/sql-gateway/rest.md| 18 ++- .../shortcodes/generated/rest_v1_dispatcher.html | 66 +- .../shortcodes/generated/rest_v1_sql_gateway.html | 14 +- ...1_sql_gateway.html => rest_v2_sql_gateway.html} | 142 +++-- docs/static/generated/rest_v1_sql_gateway.yml | 48 +-- ..._v1_sql_gateway.yml => rest_v2_sql_gateway.yml} | 111 ++-- 7 files changed, 365 insertions(+), 52 deletions(-) diff --git a/docs/content.zh/docs/dev/table/sql-gateway/rest.md b/docs/content.zh/docs/dev/table/sql-gateway/rest.md index 3ea420a21ba..3d9d6152300 100644 --- a/docs/content.zh/docs/dev/table/sql-gateway/rest.md +++ b/docs/content.zh/docs/dev/table/sql-gateway/rest.md @@ -94,7 +94,12 @@ Endpoint Options REST API -[OpenAPI specification]({{< ref_static "generated/rest_v1_sql_gateway.yml" >}}) +The available OpenAPI specification is as follows. The default version is v2. + +| Version | Description | +| --- | --- | +| [Open API v1 specification]({{< ref_static "generated/rest_v1_sql_gateway.yml" >}}) | Allow users to submit statements to the gateway and execute. | +| [Open API v2 specification]({{< ref_static "generated/rest_v2_sql_gateway.yml" >}}) | Supports SQL Client to connect to the gateway | {{< hint warning >}} The OpenAPI specification is still experimental. @@ -103,6 +108,11 @@ The OpenAPI specification is still experimental. API reference {{< tabs "f00ed142-b05f-44f0-bafc-799080c1d40d" >}} +{{< tab "v2" >}} + +{{< generated/rest_v2_sql_gateway >}} + +{{< /tab >}} {{< tab "v1" >}} {{< generated/rest_v1_sql_gateway >}} @@ -113,7 +123,9 @@ The OpenAPI specification is still experimental. Data Type Mapping -Currently, REST endpoint uses JSON format to serialize the Table Objects. Please refer -[JSON format]({{< ref "docs/connectors/table/formats/json#data-type-mapping" >}}) to the mappings. +Currently, REST endpoint supports to serialize the `RowData` with query parameter `rowFormat`. REST endpoint uses JSON format to serialize +the Table Objects. Please refer [JSON format]({{< ref "docs/connectors/table/formats/json#data-type-mapping" >}}) to the mappings. + +REST endpoint also supports to serialize the `RowData` with `PLAIN_TEXT` format that automatically cast all columns to the `String`. {{< top >}} diff --git a/docs/content/docs/dev/table/sql-gateway/rest.md b/docs/content/docs/dev/table/sql-gateway/rest.md index da44cc501bf..a38933f607c 100644 --- a/docs/content/docs/dev/table/sql-gateway/rest.md +++ b/docs/content/docs/dev/table/sql-gateway/rest.md @@ -94,7 +94,12 @@ Endpoint Options REST API -[OpenAPI specification]({{< ref_static "generated/rest_v1_sql_gateway.yml" >}}) +The available OpenAPI specification is as follows. The default version is v2. + +| Version | Description | +| --- | --- | +| [Open API v1 specification]({{< ref_static "generated/rest_v1_sql_gateway.yml" >}}) | Allow users to submit statements to the gateway and execute. | +| [Open API v2 specification]({{< ref_static "generated/rest_v2_sql_gateway.yml" >}}) | Supports SQL Client to connect to the gateway. | {{< hint warning >}} The OpenAPI specification is still experimental. @@ -103,6 +108,11 @@ The OpenAPI specification is still experimental. API reference {{< tabs "f00ed142-b05f-44f0-bafc-799080c1d40d" >}} +{{< tab "v2" >}} + +{{< generated/rest_v2_sql_gateway >}} + +{{< /tab >}} {{< tab "v1" >}} {{< generated/rest_v1_sql_gateway >}} @@ -113,7 +123,9 @@ The OpenAPI specification is still experimental. Data Type Mapping -Currently, REST endpoint uses JSON format to serialize the Table Objects. Please refer -[JSON format]({{< ref "docs/connectors/table/formats/json#data-type-mapping" >}}) to the mappings. +Currently, REST endpoint supports to serialize the `RowData` with query parameter `rowFormat`. REST endpoint uses JSON format to serialize +the Table Objects. Please refer [JSON format]({{< ref "docs/connectors/table/formats/json#data-type-mapping" >}}) to the mappings. + +REST endpoint also supports to seria
[flink] branch master updated (a6de5f23a30 -> 76a6edc7ff2)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from a6de5f23a30 [FLINK-30910][runtime] Making test wait for stop to be called before finishing the bootstrap operation (#21869) add bd33e02547e [FLINK-30925][docs] Update sql-gateway open api doc add 76a6edc7ff2 [FLINK-30925][docs] Add docs about sql-client remote mode No new revisions were added by this update. Summary of changes: docs/content.zh/docs/dev/table/sql-gateway/rest.md | 18 +- docs/content.zh/docs/dev/table/sqlClient.md| 297 docs/content/docs/dev/table/sql-gateway/rest.md| 18 +- docs/content/docs/dev/table/sqlClient.md | 298 + .../shortcodes/generated/rest_v1_dispatcher.html | 66 - .../shortcodes/generated/rest_v1_sql_gateway.html | 14 +- ...1_sql_gateway.html => rest_v2_sql_gateway.html} | 142 +- docs/static/generated/rest_v1_sql_gateway.yml | 48 +++- ..._v1_sql_gateway.yml => rest_v2_sql_gateway.yml} | 111 +++- 9 files changed, 728 insertions(+), 284 deletions(-) copy docs/layouts/shortcodes/generated/{rest_v1_sql_gateway.html => rest_v2_sql_gateway.html} (81%) copy docs/static/generated/{rest_v1_sql_gateway.yml => rest_v2_sql_gateway.yml} (81%)
[flink] branch release-1.17 updated: [FLINK-22320][docs][table] Add documentation for new introduced ALTER TABLE statements
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch release-1.17 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.17 by this push: new d2e927dc68d [FLINK-22320][docs][table] Add documentation for new introduced ALTER TABLE statements d2e927dc68d is described below commit d2e927dc68d8d31d50fbab876398adf9fb137537 Author: Jane Chan AuthorDate: Wed Jan 11 17:54:42 2023 +0800 [FLINK-22320][docs][table] Add documentation for new introduced ALTER TABLE statements This closes #21647 --- docs/content.zh/docs/dev/table/sql/alter.md | 312 ++-- docs/content/docs/dev/table/sql/alter.md| 305 +-- 2 files changed, 589 insertions(+), 28 deletions(-) diff --git a/docs/content.zh/docs/dev/table/sql/alter.md b/docs/content.zh/docs/dev/table/sql/alter.md index 63b40bb79c0..61a87858ec1 100644 --- a/docs/content.zh/docs/dev/table/sql/alter.md +++ b/docs/content.zh/docs/dev/table/sql/alter.md @@ -78,8 +78,26 @@ tableEnv.executeSql("CREATE TABLE Orders (`user` BIGINT, product STRING, amount String[] tables = tableEnv.listTables(); // or tableEnv.executeSql("SHOW TABLES").print(); -// 把 “Orders” 的表名改为 “NewOrders” -tableEnv.executeSql("ALTER TABLE Orders RENAME TO NewOrders;"); +// 新增列 `order` 并置于第一位 +tableEnv.executeSql("ALTER TABLE Orders ADD `order` INT COMMENT 'order identifier' FIRST"); + +// 新增更多列, 以及主键和 watermark +tableEnv.executeSql("ALTER TABLE Orders ADD (ts TIMESTAMP(3), category STRING AFTER product, PRIMARY KEY(`order`) NOT ENFORCED, WATERMARK FOR ts AS ts - INTERVAL '1' HOUR)"); + +// 修改列类型, 注释及 watermark 策略 +tableEnv.executeSql("ALTER TABLE Orders MODIFY (amount DOUBLE NOT NULL, category STRING COMMENT 'category identifier' AFTER `order`, WATERMARK FOR ts AS ts)"); + +// 删除 watermark +tableEnv.executeSql("ALTER TABLE Orders DROP WATERMARK"); + +// 删除列 +tableEnv.executeSql("ALTER TABLE Orders DROP (amount, ts, category)"); + +// 重命名列 +tableEnv.executeSql("ALTER TABLE Orders RENAME `order` TO order_id"); + +// "Orders" 的表名改为 "NewOrders" +tableEnv.executeSql("ALTER TABLE Orders RENAME TO NewOrders"); // 字符串数组:["NewOrders"] String[] tables = tableEnv.listTables(); @@ -93,12 +111,30 @@ val tableEnv = TableEnvironment.create(...) // 注册名为 “Orders” 的表 tableEnv.executeSql("CREATE TABLE Orders (`user` BIGINT, product STRING, amount INT) WITH (...)") +// 新增列 `order` 并置于第一位 +tableEnv.executeSql("ALTER TABLE Orders ADD `order` INT COMMENT 'order identifier' FIRST") + +// 新增更多列, 以及主键和 watermark +tableEnv.executeSql("ALTER TABLE Orders ADD (ts TIMESTAMP(3), category STRING AFTER product, PRIMARY KEY(`order`) NOT ENFORCED, WATERMARK FOR ts AS ts - INTERVAL '1' HOUR)") + +// 修改列类型, 注释, 以及主键和 watermark +tableEnv.executeSql("ALTER TABLE Orders MODIFY (amount DOUBLE NOT NULL, category STRING COMMENT 'category identifier' AFTER `order`, WATERMARK FOR ts AS ts)") + +// 删除 watermark +tableEnv.executeSql("ALTER TABLE Orders DROP WATERMARK") + +// 删除列 +tableEnv.executeSql("ALTER TABLE Orders DROP (amount, ts, category)") + +// 重命名列 +tableEnv.executeSql("ALTER TABLE Orders RENAME `order` TO order_id") + // 字符串数组: ["Orders"] val tables = tableEnv.listTables() // or tableEnv.executeSql("SHOW TABLES").print() -// 把 “Orders” 的表名改为 “NewOrders” -tableEnv.executeSql("ALTER TABLE Orders RENAME TO NewOrders;") +// rename "Orders" to "NewOrders" +tableEnv.executeSql("ALTER TABLE Orders RENAME TO NewOrders") // 字符串数组:["NewOrders"] val tables = tableEnv.listTables() @@ -113,8 +149,26 @@ table_env = TableEnvironment.create(...) tables = table_env.list_tables() # or table_env.execute_sql("SHOW TABLES").print() -# 把 “Orders” 的表名改为 “NewOrders” -table_env.execute_sql("ALTER TABLE Orders RENAME TO NewOrders;") +# 新增列 `order` 并置于第一位 +table_env.execute_sql("ALTER TABLE Orders ADD `order` INT COMMENT 'order identifier' FIRST"); + +# 新增更多列, 主键及 watermark +table_env.execute_sql("ALTER TABLE Orders ADD (ts TIMESTAMP(3), category STRING AFTER product, PRIMARY KEY(`order`) NOT ENFORCED, WATERMARK FOR ts AS ts - INTERVAL '1' HOUR)"); + +# 修改列类型, 列注释, 主键及 watermark +table_env.execute_sql("ALTER TABLE Orders MODIFY (amount DOUBLE NOT NULL, category STRING COMMENT 'category identifier' AFTER `order`, WATERMARK FOR ts AS ts)"); + +# 删除 watermark +table_env.execute_sql("ALTER TABLE Orders DROP WATERMARK"); + +# 删除列 +table_env.execute_sql("ALTER TABLE Orders DROP (amount, ts, category)"); + +# 重命名列 +table_env.ex
[flink] branch master updated (10ced269a8e -> c22c2cd55ef)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 10ced269a8e [FLINK-30901][runtime] Fix the jobVertex's parallelismConfigured is incorrect when chaining with source operators add c22c2cd55ef [FLINK-22320][docs][table] Add documentation for new introduced ALTER TABLE statements No new revisions were added by this update. Summary of changes: docs/content.zh/docs/dev/table/sql/alter.md | 312 ++-- docs/content/docs/dev/table/sql/alter.md| 305 +-- 2 files changed, 589 insertions(+), 28 deletions(-)
[flink] branch master updated: [FLINK-30811][sql-gateway] Fix "STOP JOB WITH SAVEPOINT" cannot read savepoint dir from session configuration
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new bbd5a7876eb [FLINK-30811][sql-gateway] Fix "STOP JOB WITH SAVEPOINT" cannot read savepoint dir from session configuration bbd5a7876eb is described below commit bbd5a7876eb1542ff89f05f5f5d82bb8bd41b7bd Author: Jane Chan AuthorDate: Tue Jan 31 15:54:55 2023 +0800 [FLINK-30811][sql-gateway] Fix "STOP JOB WITH SAVEPOINT" cannot read savepoint dir from session configuration This closes #21802 --- .../table/client/gateway/ExecutorImplITCase.java | 57 ++-- .../gateway/service/context/SessionContext.java| 122 ++-- .../service/operation/OperationExecutor.java | 158 ++--- .../table/gateway/service/session/Session.java | 2 +- .../service/context/SessionContextTest.java| 55 +++ 5 files changed, 218 insertions(+), 176 deletions(-) diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/ExecutorImplITCase.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/ExecutorImplITCase.java index 47634410839..770c4ea6f61 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/ExecutorImplITCase.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/ExecutorImplITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.table.client.gateway; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.CheckpointingOptions; @@ -39,7 +40,6 @@ import org.apache.flink.table.client.config.ResultMode; import org.apache.flink.table.client.gateway.result.ChangelogCollectResult; import org.apache.flink.table.client.gateway.result.MaterializedResult; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; import org.apache.flink.table.functions.AggregateFunction; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.gateway.api.operation.OperationHandle; @@ -74,6 +74,8 @@ import org.junit.jupiter.api.Order; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import javax.annotation.Nullable; @@ -184,7 +186,6 @@ class ExecutorImplITCase { config.set(StateBackendOptions.STATE_BACKEND, "hashmap"); config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); config.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, tempFolder.toURI().toString()); -config.set(CheckpointingOptions.SAVEPOINT_DIRECTORY, tempFolder.toURI().toString()); return config; } @@ -434,8 +435,9 @@ class ExecutorImplITCase { } } -@Test -void testStopJob() throws Exception { +@ValueSource(booleans = {true, false}) +@ParameterizedTest +void testStopJob(boolean withSavepoint) throws Exception { final Map configMap = new HashMap<>(); configMap.put(EXECUTION_RESULT_MODE.key(), ResultMode.TABLE.name()); configMap.put(RUNTIME_MODE.key(), RuntimeExecutionMode.STREAMING.name()); @@ -449,6 +451,7 @@ class ExecutorImplITCase { createRestServiceExecutor( Collections.singletonList(udfDependency), Configuration.fromMap(configMap))) { + executor.configureSession(srcDdl); executor.configureSession(snkDdl); StatementResult result = executor.executeStatement(insert); @@ -456,17 +459,41 @@ class ExecutorImplITCase { // wait till the job turns into running status or the test times out TestUtils.waitUntilAllTasksAreRunning(clusterClient, jobID); -StringData savepointPath = -CollectionUtil.iteratorToList( -executor.executeStatement( -String.format("STOP JOB '%s' WITH SAVEPOINT", jobID))) -.get(0) -.getString(0); -assertThat(savepointPath) -.isNotNull() -.matches( -stringData -> - Files.exists(Paths.get(URI.create(stringData.toString(); + +JobStatus expectedJobStatus; +if (withSavepoint) { +
[flink] branch master updated: [FLINK-30868][table-api] Fix to use origin LongSerializer for serialization in the TimeIndicatorTypeInfo (#21827)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new 48a2e5a4400 [FLINK-30868][table-api] Fix to use origin LongSerializer for serialization in the TimeIndicatorTypeInfo (#21827) 48a2e5a4400 is described below commit 48a2e5a440067bcab429fd9cf90e3888e0498473 Author: Shengkai <33114724+fsk...@users.noreply.github.com> AuthorDate: Mon Feb 6 15:13:15 2023 +0800 [FLINK-30868][table-api] Fix to use origin LongSerializer for serialization in the TimeIndicatorTypeInfo (#21827) --- .../flink/api/java/io/CollectionInputFormat.java | 5 +-- .../table/typeutils/TimeIndicatorTypeInfo.java | 10 +++-- .../apache/flink/table/utils/TableSchemaUtils.java | 17 + .../flink/table/utils/TableSchemaUtilsTest.java| 44 ++ .../table/planner/catalog/CatalogSchemaTable.java | 18 - .../planner/plan/FlinkCalciteCatalogReader.java| 20 +- .../plan/schema/LegacyCatalogSourceTable.scala | 13 +-- 7 files changed, 116 insertions(+), 11 deletions(-) diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/CollectionInputFormat.java b/flink-java/src/main/java/org/apache/flink/api/java/io/CollectionInputFormat.java index b85f70507b1..5fe69e870ff 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/io/CollectionInputFormat.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/io/CollectionInputFormat.java @@ -43,9 +43,8 @@ public class CollectionInputFormat extends GenericInputFormat implements N private TypeSerializer serializer; -private transient Collection -dataSet; // input data as collection. transient, because it will be serialized in a -// custom way +// input data as collection. transient, because it will be serialized in a custom way +private transient Collection dataSet; private transient Iterator iterator; diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/typeutils/TimeIndicatorTypeInfo.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/typeutils/TimeIndicatorTypeInfo.java index 827b8047181..8020f0fcfb7 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/typeutils/TimeIndicatorTypeInfo.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/typeutils/TimeIndicatorTypeInfo.java @@ -22,7 +22,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.base.LocalDateTimeSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; import org.apache.flink.api.common.typeutils.base.SqlTimestampComparator; import org.apache.flink.api.common.typeutils.base.SqlTimestampSerializer; import org.apache.flink.table.api.DataTypes; @@ -30,7 +30,8 @@ import org.apache.flink.table.api.DataTypes; import java.sql.Timestamp; /** - * Type information for indicating event or processing time. + * Type information for indicating event or processing time. However, it behaves like a regular SQL + * timestamp but is serialized as Long. * * @deprecated This class will be removed in future versions as it is used for the old type system. * It is recommended to use {@link DataTypes} instead. Please make sure to use either the old or @@ -61,10 +62,13 @@ public class TimeIndicatorTypeInfo extends SqlTimeTypeInfo { this.isEventTime = isEventTime; } +// this replaces the effective serializer by a LongSerializer +// it is a hacky but efficient solution to keep the object creation overhead low but still +// be compatible with the corresponding SqlTimestampTypeInfo @Override @SuppressWarnings("unchecked") public TypeSerializer createSerializer(ExecutionConfig executionConfig) { -return (TypeSerializer) LocalDateTimeSerializer.INSTANCE; +return (TypeSerializer) LongSerializer.INSTANCE; } public boolean isEventTime() { diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/TableSchemaUtils.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/TableSchemaUtils.java index 0ab5fc5a6f4..c23c68664f3 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/TableSchemaUtils.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/TableSchemaUtils.java @@ -24,6 +24,7 @@ import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.api.Waterm
[flink] branch master updated (111342f37bd -> 152edf6a5e6)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 111342f37bd [FLINK-30876][table-planner] Fix ResetTransformationProcessor don't reset the transformations in BatchExecMultiInput add 152edf6a5e6 [FLINK-30840][sql-client] Fix resource leak when creating Executor (#21800) No new revisions were added by this update. Summary of changes: .../org/apache/flink/table/client/SqlClient.java | 36 ++-- .../flink/table/client/gateway/Executor.java | 23 +-- .../flink/table/client/gateway/ExecutorImpl.java | 142 +--- .../flink/table/client/cli/CliClientITCase.java| 7 +- .../flink/table/client/cli/CliClientTest.java | 37 + .../table/client/gateway/ExecutorImplITCase.java | 181 ++--- 6 files changed, 204 insertions(+), 222 deletions(-)
[flink] branch master updated (fd4f918bbaa -> 615fde6667a)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from fd4f918bbaa [FLINK-30474][runtime] Propagates leader information change only if the leadership is still acquired add 615fde6667a [FLINK-30827][sql-client] Improve the SQL Client code structure (#21790) No new revisions were added by this update. Summary of changes: .../org/apache/flink/table/client/SqlClient.java | 4 ++-- .../table/client/cli/CliChangelogResultView.java | 2 +- .../flink/table/client/cli/CliTableResultView.java | 2 +- .../table/client/cli/CliTableauResultView.java | 2 +- .../gateway/{local => }/DefaultContextUtils.java | 2 +- .../table/client/gateway/ResultDescriptor.java | 8 +++ .../gateway/{local => }/SingleSessionManager.java | 2 +- .../{local => }/result/ChangelogCollectResult.java | 2 +- .../{local => }/result/ChangelogResult.java| 2 +- .../{local => }/result/CollectResultBase.java | 2 +- .../gateway/{local => }/result/DynamicResult.java | 2 +- .../result/MaterializedCollectBatchResult.java | 2 +- .../result/MaterializedCollectResultBase.java | 2 +- .../result/MaterializedCollectStreamResult.java| 2 +- .../{local => }/result/MaterializedResult.java | 2 +- .../flink/table/client/cli/CliClientITCase.java| 2 +- .../flink/table/client/cli/CliResultViewTest.java | 4 ++-- .../table/client/cli/CliTableauResultViewTest.java | 2 +- .../gateway/{local => }/ExecutorImplITCase.java| 25 -- .../result/BaseMaterializedResultTest.java | 2 +- .../result/ChangelogCollectResultTest.java | 2 +- .../result/MaterializedCollectBatchResultTest.java | 2 +- .../MaterializedCollectStreamResultTest.java | 2 +- 23 files changed, 36 insertions(+), 43 deletions(-) rename flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/{local => }/DefaultContextUtils.java (98%) rename flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/{local => }/SingleSessionManager.java (99%) rename flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/{local => }/result/ChangelogCollectResult.java (98%) rename flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/{local => }/result/ChangelogResult.java (95%) rename flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/{local => }/result/CollectResultBase.java (98%) rename flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/{local => }/result/DynamicResult.java (94%) rename flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/{local => }/result/MaterializedCollectBatchResult.java (97%) rename flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/{local => }/result/MaterializedCollectResultBase.java (98%) rename flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/{local => }/result/MaterializedCollectStreamResult.java (98%) rename flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/{local => }/result/MaterializedResult.java (95%) rename flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/{local => }/ExecutorImplITCase.java (97%) rename flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/{local => }/result/BaseMaterializedResultTest.java (97%) rename flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/{local => }/result/ChangelogCollectResultTest.java (98%) rename flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/{local => }/result/MaterializedCollectBatchResultTest.java (99%) rename flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/{local => }/result/MaterializedCollectStreamResultTest.java (99%)
[flink] branch master updated: [FLINK-30819][sql-client] Fix sql client print an empty line between the multi-line statements
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new db82f82bd94 [FLINK-30819][sql-client] Fix sql client print an empty line between the multi-line statements db82f82bd94 is described below commit db82f82bd94a0af059bcfd6e85b3ae31bdc5c76e Author: Shengkai <1059623...@qq.com> AuthorDate: Mon Jan 30 18:20:31 2023 +0800 [FLINK-30819][sql-client] Fix sql client print an empty line between the multi-line statements This closes #21786 --- .../apache/flink/table/client/cli/CliClient.java | 217 +++--- .../org/apache/flink/table/client/cli/Printer.java | 253 + .../client/cli/parser/SqlMultiLineParser.java | 71 +- .../flink/table/client/gateway/Executor.java | 2 +- .../flink/table/client/gateway/ExecutorImpl.java | 8 +- .../table/client/gateway/ResultDescriptor.java | 4 +- .../{ClientResult.java => StatementResult.java}| 6 +- .../local/result/ChangelogCollectResult.java | 4 +- .../gateway/local/result/CollectResultBase.java| 4 +- .../result/MaterializedCollectBatchResult.java | 6 +- .../result/MaterializedCollectResultBase.java | 4 +- .../result/MaterializedCollectStreamResult.java| 6 +- .../apache/flink/table/client/SqlClientTest.java | 57 - .../flink/table/client/cli/CliClientTest.java | 14 +- .../client/gateway/local/ExecutorImplITCase.java | 6 +- .../local/result/ChangelogCollectResultTest.java | 4 +- .../result/MaterializedCollectBatchResultTest.java | 8 +- .../MaterializedCollectStreamResultTest.java | 8 +- .../table/client/util/CliClientTestUtils.java | 6 +- 19 files changed, 449 insertions(+), 239 deletions(-) diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java index cf41ffc85cd..bf522234b77 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java @@ -19,19 +19,12 @@ package org.apache.flink.table.client.cli; import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.JobID; -import org.apache.flink.table.api.ResultKind; -import org.apache.flink.table.api.SqlParserEOFException; import org.apache.flink.table.client.SqlClientException; -import org.apache.flink.table.client.cli.parser.Command; import org.apache.flink.table.client.cli.parser.SqlCommandParserImpl; import org.apache.flink.table.client.cli.parser.SqlMultiLineParser; import org.apache.flink.table.client.config.SqlClientOptions; -import org.apache.flink.table.client.gateway.ClientResult; import org.apache.flink.table.client.gateway.Executor; -import org.apache.flink.table.client.gateway.ResultDescriptor; import org.apache.flink.table.client.gateway.SqlExecutionException; -import org.apache.flink.table.utils.print.PrintStyle; import org.jline.reader.EndOfFileException; import org.jline.reader.LineReader; @@ -41,7 +34,6 @@ import org.jline.reader.UserInterruptException; import org.jline.terminal.Terminal; import org.jline.utils.AttributedStringBuilder; import org.jline.utils.AttributedStyle; -import org.jline.utils.InfoCmp; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -57,14 +49,6 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.function.Supplier; -import static org.apache.flink.table.api.config.TableConfigOptions.TABLE_DML_SYNC; -import static org.apache.flink.table.client.cli.CliStrings.MESSAGE_EXECUTE_STATEMENT; -import static org.apache.flink.table.client.cli.CliStrings.MESSAGE_FINISH_STATEMENT; -import static org.apache.flink.table.client.cli.CliStrings.MESSAGE_STATEMENT_SUBMITTED; -import static org.apache.flink.table.client.cli.CliStrings.MESSAGE_SUBMITTING_STATEMENT; -import static org.apache.flink.table.client.config.ResultMode.TABLEAU; -import static org.apache.flink.table.client.config.SqlClientOptions.EXECUTION_RESULT_MODE; - /** SQL CLI client. */ public class CliClient implements AutoCloseable { @@ -91,8 +75,6 @@ public class CliClient implements AutoCloseable { private boolean isRunning; -private final SqlMultiLineParser parser; - /** * Creates a CLI instance with a custom terminal. Make sure to close the CLI instance afterwards * using {@link #close()}. @@ -107,7 +89,6 @@ public class CliClient implements AutoCloseable { this.executor = executor; this.inputTransformer = inputTransformer; this.historyFilePath = historyFilePath; -this.parser = new SqlMultiLineParser(new SqlCommandP
[flink] branch master updated: [FLINK-30538][sql-client] Fix OperationExecutor throw duplicate exception when meeet error
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new d657395248c [FLINK-30538][sql-client] Fix OperationExecutor throw duplicate exception when meeet error d657395248c is described below commit d657395248c4cb189ee11356aa596dd84093b3d3 Author: Paul Lin AuthorDate: Wed Feb 1 14:34:36 2023 +0800 [FLINK-30538][sql-client] Fix OperationExecutor throw duplicate exception when meeet error This closes #21581 --- .../service/operation/OperationExecutor.java | 64 ++ 1 file changed, 30 insertions(+), 34 deletions(-) diff --git a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java index de030c4f033..9c11f3a5d55 100644 --- a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java +++ b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java @@ -489,45 +489,41 @@ public class OperationExecutor { Duration clientTimeout = Configuration.fromMap(sessionContext.getConfigMap()) .get(ClientOptions.CLIENT_TIMEOUT); -Optional savepoint; -try { -savepoint = -runClusterAction( -handle, -clusterClient -> { +Optional savepoint = +runClusterAction( +handle, +clusterClient -> { +try { if (isWithSavepoint) { // blocking get savepoint path -try { -return Optional.of( -clusterClient -.stopWithSavepoint( - JobID.fromHexString(jobId), -isWithDrain, - executionConfig.get( - CheckpointingOptions - .SAVEPOINT_DIRECTORY), - SavepointFormatType.DEFAULT) -.get( - clientTimeout.toMillis(), - TimeUnit.MILLISECONDS)); -} catch (Exception e) { -throw new FlinkException( -"Could not stop job " -+ stopJobOperation.getJobId() -+ " in session " -+ handle.getIdentifier() -+ ".", -e); -} +return Optional.of( +clusterClient +.stopWithSavepoint( + JobID.fromHexString(jobId), +isWithDrain, + executionConfig.get( + CheckpointingOptions + .SAVEPOINT_DIRECTORY), + SavepointFormatType.DEFAULT) +.get( + clientTimeout.toMillis(), + TimeUnit.MILLISECONDS)); } else { - clusterClient.cancel(JobID.fromHexString(jobId)); +clusterClient +.cancel(JobID.fromHexString(jobId)) +.get(clientTimeou
[flink] branch master updated (9e8a99c12ce -> 009a9e5ce7d)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 9e8a99c12ce [FLINK-24456][Connectors / Kafka,Table SQL / Ecosystem] Support bounded offset in the Kafka table connector add 009a9e5ce7d [FLINK-30848][table-planner] Enable CatalogTableITCase#testAlterTable (#21803) No new revisions were added by this update. Summary of changes: .../apache/flink/table/planner/catalog/CatalogTableITCase.scala| 7 +++ 1 file changed, 3 insertions(+), 4 deletions(-)
[flink] branch master updated (5f5abb69187 -> c9e87fe410c)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 5f5abb69187 Revert "[FLINK-19699][e2e] Collect dmesg output after tests" add c9e87fe410c [FLINK-30692][sql-client] Introduce SingleSessionManager to support REMOVE JAR syntax (#21771) No new revisions were added by this update. Summary of changes: .../endpoint/hive/HiveServer2EndpointITCase.java | 5 +- .../org/apache/flink/table/client/SqlClient.java | 46 +++-- .../apache/flink/table/client/cli/CliStrings.java | 39 .../flink/table/client/gateway/Executor.java | 10 + .../flink/table/client/gateway/ExecutorImpl.java | 6 +- .../client/gateway/local/DefaultContextUtils.java | 26 +-- .../client/gateway/local/SingleSessionManager.java | 186 ++ .../flink/table/client/cli/CliClientITCase.java| 4 +- .../client/gateway/local/ExecutorImplITCase.java | 4 +- .../src/test/resources/sql/function.q | 28 +-- .../flink-sql-client/src/test/resources/sql/set.q | 30 ++- .../org/apache/flink/table/gateway/SqlGateway.java | 53 ++ .../gateway/service/context/DefaultContext.java| 31 ++- .../gateway/service/context/SessionContext.java| 67 --- .../service/operation/OperationExecutor.java | 12 +- .../gateway/service/session/SessionManager.java| 207 +++-- ...SessionManager.java => SessionManagerImpl.java} | 38 ++-- .../gateway/service/SqlGatewayServiceITCase.java | 6 +- ...anagerTest.java => SessionManagerImplTest.java} | 8 +- .../service/utils/SqlGatewayServiceExtension.java | 16 +- 20 files changed, 419 insertions(+), 403 deletions(-) create mode 100644 flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/SingleSessionManager.java copy flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/session/{SessionManager.java => SessionManagerImpl.java} (90%) rename flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/session/{SessionManagerTest.java => SessionManagerImplTest.java} (94%)
[flink] branch master updated: [FLINK-30816][sql-client] Fix SQL Client always uses the highest packaged version to connect to gateway
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new cc6315a3e82 [FLINK-30816][sql-client] Fix SQL Client always uses the highest packaged version to connect to gateway cc6315a3e82 is described below commit cc6315a3e82c570e631684793d08ca9cb5403521 Author: Shengkai <1059623...@qq.com> AuthorDate: Sun Jan 29 18:29:27 2023 +0800 [FLINK-30816][sql-client] Fix SQL Client always uses the highest packaged version to connect to gateway This closes #21776 --- .../flink/table/client/gateway/ExecutorImpl.java | 85 +++--- .../client/gateway/local/ExecutorImplITCase.java | 68 + .../rest/handler/util/GetApiVersionHandler.java| 25 +-- .../rest/header/statement/FetchResultsHeaders.java | 4 +- .../rest/util/SqlGatewayRestAPIVersion.java| 16 5 files changed, 179 insertions(+), 19 deletions(-) diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/ExecutorImpl.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/ExecutorImpl.java index 93e9550224e..f324f516406 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/ExecutorImpl.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/ExecutorImpl.java @@ -29,6 +29,7 @@ import org.apache.flink.runtime.rest.messages.MessageParameters; import org.apache.flink.runtime.rest.messages.RequestBody; import org.apache.flink.runtime.rest.messages.ResponseBody; import org.apache.flink.runtime.rest.util.RestClientException; +import org.apache.flink.runtime.rest.versioning.RestAPIVersion; import org.apache.flink.table.api.SqlParserEOFException; import org.apache.flink.table.client.SqlClientException; import org.apache.flink.table.data.RowData; @@ -46,6 +47,7 @@ import org.apache.flink.table.gateway.rest.header.session.TriggerSessionHeartbea import org.apache.flink.table.gateway.rest.header.statement.CompleteStatementHeaders; import org.apache.flink.table.gateway.rest.header.statement.ExecuteStatementHeaders; import org.apache.flink.table.gateway.rest.header.statement.FetchResultsHeaders; +import org.apache.flink.table.gateway.rest.header.util.GetApiVersionHeaders; import org.apache.flink.table.gateway.rest.message.operation.OperationMessageParameters; import org.apache.flink.table.gateway.rest.message.operation.OperationStatusResponseBody; import org.apache.flink.table.gateway.rest.message.session.CloseSessionResponseBody; @@ -61,9 +63,11 @@ import org.apache.flink.table.gateway.rest.message.statement.FetchResultsMessage import org.apache.flink.table.gateway.rest.message.statement.FetchResultsResponseBody; import org.apache.flink.table.gateway.rest.serde.ResultInfo; import org.apache.flink.table.gateway.rest.util.RowFormat; +import org.apache.flink.table.gateway.rest.util.SqlGatewayRestAPIVersion; import org.apache.flink.table.gateway.rest.util.SqlGatewayRestEndpointUtils; import org.apache.flink.table.gateway.service.context.DefaultContext; import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -72,6 +76,7 @@ import javax.annotation.Nullable; import java.io.IOException; import java.net.InetSocketAddress; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.UUID; @@ -83,6 +88,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.function.Function; import java.util.function.Supplier; +import java.util.stream.Collectors; import static org.apache.flink.table.gateway.rest.handler.session.CloseSessionHandler.CLOSE_MESSAGE; @@ -97,9 +103,10 @@ public class ExecutorImpl implements Executor { private final long heartbeatInterval; private final ExecutorService service; private final ScheduledExecutorService heartbeatScheduler; +private final RestClient restClient; -private RestClient restClient; private SessionHandle sessionHandle; +private SqlGatewayRestAPIVersion connectionVersion; @VisibleForTesting public ExecutorImpl( @@ -111,6 +118,11 @@ public class ExecutorImpl implements Executor { this.heartbeatInterval = heartbeatInterval; this.service = Executors.newCachedThreadPool(); this.heartbeatScheduler = Executors.newSingleThreadScheduledExecutor(); +try { +this.restClient = new RestClient(defaultContext.getFlinkConfig(), service); +} catch (Exception e) { +throw new SqlClientException("Can not create the Rest Client.", e); +} } public ExecutorImpl(DefaultContex
[flink] branch master updated (6e445064f6a -> 95df4e726f3)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 6e445064f6a [FLINK-30799][runtime] Make SinkFunction support speculative execution through implementing SupportsConcurrentExecutionAttempts interface add 0b783a54e8d [FLINK-30691][sql-client] Supports the gateway mode in the sql-client add 95df4e726f3 [FLINK-30691][e2e] Add E2E test for the sql-client gateway mode No new revisions were added by this update. Summary of changes: .../flink/tests/util/flink/FlinkDistribution.java | 96 ++- .../flink-sql-gateway-test/pom.xml | 6 + .../flink/table/gateway/SqlGatewayE2ECase.java | 29 +++- .../org/apache/flink/table/client/SqlClient.java | 133 ++-- .../apache/flink/table/client/cli/CliOptions.java | 82 +++--- .../flink/table/client/cli/CliOptionsParser.java | 159 --- ...lContextUtils.java => DefaultContextUtils.java} | 15 +- .../apache/flink/table/client/SqlClientTest.java | 40 - .../src/test/resources/cli/all-mode-help.out | 175 + .../src/test/resources/cli/embedded-mode-help.out | 136 .../src/test/resources/cli/gateway-mode-help.out | 36 + .../apache/flink/test/util/SQLJobClientMode.java | 97 .../apache/flink/test/util/SQLJobSubmission.java | 19 +-- pom.xml| 2 +- 14 files changed, 785 insertions(+), 240 deletions(-) rename flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/{LocalContextUtils.java => DefaultContextUtils.java} (86%) create mode 100644 flink-table/flink-sql-client/src/test/resources/cli/all-mode-help.out create mode 100644 flink-table/flink-sql-client/src/test/resources/cli/embedded-mode-help.out create mode 100644 flink-table/flink-sql-client/src/test/resources/cli/gateway-mode-help.out create mode 100644 flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/SQLJobClientMode.java
[flink] branch master updated (d336dad9345 -> 1534ea75161)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from d336dad9345 [FLINK-30755][runtime] Remove legacy codes of marking not support speculative executions add 1534ea75161 [FLINK-29945][sql-client] Supports to submit SQL to an embedded SQL Gateway in the SQL Client No new revisions were added by this update. Summary of changes: .../delegation/hive/HiveOperationExecutor.java | 2 +- flink-table/flink-sql-client/pom.xml | 16 + .../org/apache/flink/table/client/SqlClient.java | 86 +++- .../apache/flink/table/client/cli/CliClient.java | 453 + .../flink/table/client/cli/CliOptionsParser.java | 7 +- .../apache/flink/table/client/cli/CliStrings.java | 12 +- .../flink/table/client/cli/SqlCompleter.java | 2 +- .../table/client/cli/parser/ClientParser.java | 199 - .../parser/{StatementType.java => Command.java}| 20 +- .../table/client/cli/parser/SqlCommandParser.java | 17 +- .../client/cli/parser/SqlCommandParserImpl.java| 137 ++- .../client/cli/parser/SqlMultiLineParser.java | 28 +- .../flink/table/client/gateway/ClientResult.java | 113 + .../flink/table/client/gateway/Executor.java | 91 + .../flink/table/client/gateway/ExecutorImpl.java | 432 .../table/client/gateway/ResultDescriptor.java | 7 +- .../client/gateway/context/DefaultContext.java | 144 --- .../client/gateway/context/ExecutionContext.java | 181 .../client/gateway/context/SessionContext.java | 305 -- .../client/gateway/local/LocalContextUtils.java| 56 +-- .../table/client/gateway/local/LocalExecutor.java | 302 -- .../local/result/ChangelogCollectResult.java | 4 +- .../gateway/local/result/CollectResultBase.java| 10 +- .../result/MaterializedCollectBatchResult.java | 6 +- .../result/MaterializedCollectResultBase.java | 4 +- .../result/MaterializedCollectStreamResult.java| 6 +- .../apache/flink/table/client/SqlClientTest.java | 2 +- .../flink/table/client/cli/CliClientITCase.java| 35 +- .../flink/table/client/cli/CliClientTest.java | 250 +++- .../flink/table/client/cli/CliResultViewTest.java | 6 +- .../table/client/cli/CliTableauResultViewTest.java | 27 +- ...rserTest.java => SqlCommandParserImplTest.java} | 74 ++-- .../table/client/cli/utils/TestTableResult.java| 130 -- .../client/gateway/context/SessionContextTest.java | 230 --- .../table/client/gateway/local/DependencyTest.java | 314 -- ...ExecutorITCase.java => ExecutorImplITCase.java} | 314 +++--- .../local/result/ChangelogCollectResultTest.java | 19 +- .../result/MaterializedCollectBatchResultTest.java | 17 +- .../MaterializedCollectStreamResultTest.java | 19 +- .../client/gateway/utils/SimpleCatalogFactory.java | 133 -- .../table/client/util/CliClientTestUtils.java | 50 +++ .../table/client/util/TestHiveCatalogFactory.java | 124 ++ .../org.apache.flink.table.factories.Factory | 5 +- .../src/test/resources/sql/catalog_database.q | 9 +- .../src/test/resources/sql/function.q | 30 +- .../src/test/resources/sql/insert.q| 12 +- .../src/test/resources/sql/module.q| 3 +- .../src/test/resources/sql/select.q| 16 +- .../flink-sql-client/src/test/resources/sql/set.q | 186 + .../src/test/resources/sql/statement_set.q | 34 +- .../src/test/resources/sql/table.q | 226 +++--- .../flink-sql-client/src/test/resources/sql/view.q | 27 +- .../src/test/resources/sql_multi/statement_set.q | 34 +- flink-table/flink-sql-gateway/pom.xml | 6 - .../org/apache/flink/table/gateway/SqlGateway.java | 27 +- .../table/gateway/rest/SqlGatewayRestEndpoint.java | 8 +- .../header/session/ConfigureSessionHeaders.java| 2 +- .../header/statement/CompleteStatementHeaders.java | 2 +- .../operation/OperationStatusResponseBody.java | 2 + .../message/session/CloseSessionResponseBody.java | 2 + .../session/ConfigureSessionRequestBody.java | 6 + .../session/GetSessionConfigResponseBody.java | 2 + .../message/session/OpenSessionRequestBody.java| 2 + .../message/session/OpenSessionResponseBody.java | 2 + .../statement/CompleteStatementRequestBody.java| 2 + .../statement/CompleteStatementResponseBody.java | 2 + .../statement/ExecuteStatementRequestBody.java | 6 + .../message/util/GetApiVersionResponseBody.java| 2 + .../rest/message/util/GetInfoResponseBody.java | 2 + .../SqlGatewayRestEndpointUtils.java} | 33 +- .../gateway/service/SqlGatewayServiceImpl.java | 1 + .../gateway/service/context/DefaultCon
[flink] branch master updated (7ea4476c054 -> 7b69e93b301)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 7ea4476c054 [FLINK-30093][protobuf] Fix compile errors for google.protobuf.Timestamp type add 7b69e93b301 [FLINK-29000][python] Support python UDF in the SQL Gateway (#21725) No new revisions were added by this update. Summary of changes: flink-dist/src/main/flink-bin/bin/config.sh| 20 - flink-dist/src/main/flink-bin/bin/flink-console.sh | 2 +- flink-dist/src/main/flink-bin/bin/flink-daemon.sh | 2 +- flink-table/flink-sql-gateway/pom.xml | 6 .../gateway/service/context/DefaultContext.java| 35 +- .../service/context/SessionContextTest.java| 5 +++- .../service/session/SessionManagerTest.java| 5 +++- 7 files changed, 63 insertions(+), 12 deletions(-)
[flink] branch master updated: [FLINK-30677][sql-gateway] Fix unstable SqlGatewayServiceStatementITCase#testFlinkSqlStatements
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new 4fdb5c40094 [FLINK-30677][sql-gateway] Fix unstable SqlGatewayServiceStatementITCase#testFlinkSqlStatements 4fdb5c40094 is described below commit 4fdb5c40094cfaa5fb3b6d7ce9ec891dab3ef32a Author: Paul Lin AuthorDate: Mon Jan 16 21:13:34 2023 +0800 [FLINK-30677][sql-gateway] Fix unstable SqlGatewayServiceStatementITCase#testFlinkSqlStatements This closes #21700 --- .../service/operation/OperationExecutor.java | 3 +- .../gateway/service/result/ResultFetcher.java | 42 +- .../gateway/AbstractSqlGatewayStatementITCase.java | 9 ++--- .../SqlGatewayRestEndpointStatementITCase.java | 4 +++ 4 files changed, 37 insertions(+), 21 deletions(-) diff --git a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java index cfcb70a059c..3c377eba893 100644 --- a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java +++ b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/operation/OperationExecutor.java @@ -409,7 +409,8 @@ public class OperationExecutor { ResolvedSchema.of(Column.physical(JOB_ID, DataTypes.STRING())), Collections.singletonList( GenericRowData.of(StringData.fromString(jobID.toString(, -jobID); +jobID, +result.getResultKind()); } private ResultFetcher callOperation( diff --git a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/result/ResultFetcher.java b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/result/ResultFetcher.java index b48659e11cd..9fa810c9c8c 100644 --- a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/result/ResultFetcher.java +++ b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/result/ResultFetcher.java @@ -20,6 +20,7 @@ package org.apache.flink.table.gateway.service.result; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; +import org.apache.flink.core.execution.JobClient; import org.apache.flink.table.api.ResultKind; import org.apache.flink.table.api.internal.TableResultInternal; import org.apache.flink.table.catalog.ResolvedSchema; @@ -31,6 +32,7 @@ import org.apache.flink.table.gateway.api.results.ResultSetImpl; import org.apache.flink.table.gateway.service.utils.SqlExecutionException; import org.apache.flink.table.utils.print.RowDataToStringConverter; import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.CollectionUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -119,7 +121,8 @@ public class ResultFetcher { OperationHandle operationHandle, ResolvedSchema resultSchema, List rows, -@Nullable JobID jobID) { +@Nullable JobID jobID, +ResultKind resultKind) { this.operationHandle = operationHandle; this.resultSchema = resultSchema; this.bufferedResults.addAll(rows); @@ -127,16 +130,15 @@ public class ResultFetcher { this.converter = SIMPLE_ROW_DATA_TO_STRING_CONVERTER; this.isQueryResult = false; this.jobID = jobID; -this.resultKind = ResultKind.SUCCESS_WITH_CONTENT; +this.resultKind = resultKind; } public static ResultFetcher fromTableResult( OperationHandle operationHandle, TableResultInternal tableResult, boolean isQueryResult) { -JobID jobID = null; if (isQueryResult) { -jobID = +JobID jobID = tableResult .getJobClient() .orElseThrow( @@ -146,29 +148,37 @@ public class ResultFetcher { "Can't get job client for the operation %s.", operationHandle))) .getJobID(); +return new ResultFetcher( +operationHandle, +tableResult.getResolvedSchema(), +tableResult.collectInternal(), +tableResult.getRowDataToStringConverter(), +true, +jobID, +tableResult.getResultKind()); +} else { +return ne
[flink] branch master updated: [FLINK-30693][sql-gateway] Supports to specify RowFormat when fetching results
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/master by this push: new 029860f26ce [FLINK-30693][sql-gateway] Supports to specify RowFormat when fetching results 029860f26ce is described below commit 029860f26ce8455cdd2bd1244785f42fee0440ca Author: yuzelin <747884...@qq.com> AuthorDate: Sun Jan 15 20:55:50 2023 +0800 [FLINK-30693][sql-gateway] Supports to specify RowFormat when fetching results This closes #21677 --- .../endpoint/hive/HiveServer2EndpointITCase.java | 4 +- .../hive/HiveServer2EndpointStatementITCase.java | 75 .../src/test/resources/endpoint/select.q | 58 +- .../table/gateway/api/results/ResultSetImpl.java | 4 + .../table/gateway/rest/SqlGatewayRestEndpoint.java | 16 ++- .../handler/statement/FetchResultsHandler.java | 54 ++--- .../rest/header/statement/FetchResultsHeaders.java | 78 ++--- .../statement/FetchResultResponseBodyImpl.java | 84 ++ ...ers.java => FetchResultsMessageParameters.java} | 30 +++-- .../statement/FetchResultsResponseBody.java| 52 +++-- .../FetchResultsRowFormatQueryParameter.java | 49 .../statement/NotReadyFetchResultResponse.java}| 48 .../serde/FetchResultResponseBodyDeserializer.java | 90 +++ .../serde/FetchResultsResponseBodySerializer.java | 67 +++ .../flink/table/gateway/rest/serde/ResultInfo.java | 126 - ...serializer.java => ResultInfoDeserializer.java} | 64 --- ...onSerializer.java => ResultInfoSerializer.java} | 100 ++-- .../flink/table/gateway/rest/util/RowFormat.java | 36 ++ .../service/operation/OperationManager.java| 5 +- .../gateway/service/result/NotReadyResult.java | 2 +- .../gateway/AbstractSqlGatewayStatementITCase.java | 63 --- .../table/gateway/rest/OperationRelatedITCase.java | 4 +- .../table/gateway/rest/RestAPIITCaseBase.java | 4 +- .../gateway/rest/SqlGatewayRestEndpointITCase.java | 4 +- .../SqlGatewayRestEndpointStatementITCase.java | 100 +++- .../gateway/rest/SqlGatewayRestEndpointTest.java | 4 +- .../table/gateway/rest/StatementRelatedITCase.java | 20 +++- .../rest/serde/ResultInfoJsonSerDeTest.java| 123 ++-- .../rest/util/SqlGatewayRestEndpointExtension.java | 4 +- ...s.java => SqlGatewayRestEndpointTestUtils.java} | 16 ++- .../gateway/service/SqlGatewayServiceITCase.java | 4 +- ...{resultInfo.txt => result_info_json_format.txt} | 2 +- .../resources/result_info_plain_text_format.txt| 1 + .../src/test/resources/sql/begin_statement_set.q | 60 +- .../src/test/resources/sql/insert.q| 80 + .../src/test/resources/sql/statement_set.q | 61 +- .../resources/sql_gateway_rest_api_v1.snapshot | 14 +-- .../resources/sql_gateway_rest_api_v2.snapshot | 21 +--- 38 files changed, 1042 insertions(+), 585 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/endpoint/hive/HiveServer2EndpointITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/endpoint/hive/HiveServer2EndpointITCase.java index 98e8612a54a..71bb5277f37 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/endpoint/hive/HiveServer2EndpointITCase.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/endpoint/hive/HiveServer2EndpointITCase.java @@ -38,6 +38,7 @@ import org.apache.flink.table.gateway.api.session.SessionEnvironment; import org.apache.flink.table.gateway.api.session.SessionHandle; import org.apache.flink.table.gateway.api.utils.SqlGatewayException; import org.apache.flink.table.gateway.service.SqlGatewayServiceImpl; +import org.apache.flink.table.gateway.service.result.NotReadyResult; import org.apache.flink.table.gateway.service.session.SessionManager; import org.apache.flink.table.gateway.service.utils.SqlGatewayServiceExtension; import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.JavaFunc0; @@ -114,7 +115,6 @@ import static org.apache.flink.table.api.config.TableConfigOptions.TABLE_DML_SYN import static org.apache.flink.table.api.config.TableConfigOptions.TABLE_SQL_DIALECT; import static org.apache.flink.table.endpoint.hive.util.ThriftObjectConversions.toSessionHandle; import static org.apache.flink.table.endpoint.hive.util.ThriftObjectConversions.toTOperationHandle; -import static org.apache.flink.table.gateway.service.result.NotReadyResult.NOT_READY_RESULT; import static org.apache.hive.service.rpc.thrift.TProtocolVersion.HIVE_CLI_SERVICE_PROTOCO
[flink] branch master updated (6cecf624552 -> 128b0c4442a)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 6cecf624552 [hotfix][docs][table] Fix broken links in determinism.md (#21643) add 128b0c4442a [FLINK-30675][sql-client] Decompose query result print from Executor No new revisions were added by this update. Summary of changes: .../table/client/cli/CliChangelogResultView.java | 55 ++--- .../apache/flink/table/client/cli/CliClient.java | 36 +-- .../flink/table/client/cli/CliInputView.java | 13 +- .../flink/table/client/cli/CliResultView.java | 19 +- .../apache/flink/table/client/cli/CliRowView.java | 13 +- .../flink/table/client/cli/CliTableResultView.java | 73 +++--- .../table/client/cli/CliTableauResultView.java | 32 +-- .../org/apache/flink/table/client/cli/CliView.java | 68 +++-- .../flink/table/client/cli/TerminalUtils.java | 6 + .../flink/table/client/gateway/Executor.java | 22 -- .../table/client/gateway/ResultDescriptor.java | 59 +++-- .../table/client/gateway/local/LocalExecutor.java | 94 +-- .../table/client/gateway/local/ResultStore.java| 95 --- .../local/result/ChangelogCollectResult.java | 5 - .../gateway/local/result/CollectResultBase.java| 17 +- .../client/gateway/local/result/DynamicResult.java | 8 +- .../result/MaterializedCollectResultBase.java | 5 - .../flink/table/client/cli/CliClientTest.java | 25 -- .../flink/table/client/cli/CliResultViewTest.java | 242 +- .../table/client/cli/CliTableauResultViewTest.java | 273 + .../flink/table/client/cli/TestingExecutor.java| 176 - .../table/client/cli/TestingExecutorBuilder.java | 55 - .../client/gateway/local/LocalExecutorITCase.java | 28 +-- 23 files changed, 395 insertions(+), 1024 deletions(-) delete mode 100644 flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ResultStore.java delete mode 100644 flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/TestingExecutor.java delete mode 100644 flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/TestingExecutorBuilder.java
[flink] branch master updated (95869e0bc99 -> 6cecf624552)
This is an automated email from the ASF dual-hosted git repository. shengkai pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 95869e0bc99 [FLINK-30579][hive] Introducing configurable option to use hive native function add 6cecf624552 [hotfix][docs][table] Fix broken links in determinism.md (#21643) No new revisions were added by this update. Summary of changes: docs/content/docs/dev/table/concepts/determinism.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-)