[flink] branch master updated: [FLINK-32052][table-runtime] Introduce left and right state retention time to StreamingJoinOperator
This is an automated email from the ASF dual-hosted git repository. godfrey 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 5ba3f2bdea6 [FLINK-32052][table-runtime] Introduce left and right state retention time to StreamingJoinOperator 5ba3f2bdea6 is described below commit 5ba3f2bdea6fc7c9e58b50200806ea341b7dd3d3 Author: Jane Chan AuthorDate: Sun Apr 30 00:29:29 2023 +0800 [FLINK-32052][table-runtime] Introduce left and right state retention time to StreamingJoinOperator This closes #22566 --- .../plan/nodes/exec/stream/StreamExecJoin.java | 2 + .../join/stream/AbstractStreamingJoinOperator.java | 9 +- .../join/stream/StreamingJoinOperator.java | 14 +- .../join/stream/StreamingSemiAntiJoinOperator.java | 12 +- .../join/stream/StreamingJoinOperatorTest.java | 656 + .../join/stream/StreamingJoinOperatorTestBase.java | 142 + .../stream/StreamingSemiAntiJoinOperatorTest.java | 294 + .../operators/sink/SinkUpsertMaterializerTest.java | 62 +- .../table/runtime/util/RowDataHarnessAssertor.java | 32 + 9 files changed, 1164 insertions(+), 59 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java index da2800d246f..47544eeb6f8 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java @@ -184,6 +184,7 @@ public class StreamExecJoin extends ExecNodeBase leftInputSpec, rightInputSpec, joinSpec.getFilterNulls(), +minRetentionTime, minRetentionTime); } else { boolean leftIsOuter = joinType == FlinkJoinType.LEFT || joinType == FlinkJoinType.FULL; @@ -199,6 +200,7 @@ public class StreamExecJoin extends ExecNodeBase leftIsOuter, rightIsOuter, joinSpec.getFilterNulls(), +minRetentionTime, minRetentionTime); } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/AbstractStreamingJoinOperator.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/AbstractStreamingJoinOperator.java index 64ada0f0db4..c7dad646631 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/AbstractStreamingJoinOperator.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/AbstractStreamingJoinOperator.java @@ -60,7 +60,8 @@ public abstract class AbstractStreamingJoinOperator extends AbstractStreamOperat private final boolean[] filterNullKeys; -protected final long stateRetentionTime; +protected final long leftStateRetentionTime; +protected final long rightStateRetentionTime; protected transient JoinConditionWithNullFilters joinCondition; protected transient TimestampedCollector collector; @@ -72,13 +73,15 @@ public abstract class AbstractStreamingJoinOperator extends AbstractStreamOperat JoinInputSideSpec leftInputSideSpec, JoinInputSideSpec rightInputSideSpec, boolean[] filterNullKeys, -long stateRetentionTime) { +long leftStateRetentionTime, +long rightStateRetentionTime) { this.leftType = leftType; this.rightType = rightType; this.generatedJoinCondition = generatedJoinCondition; this.leftInputSideSpec = leftInputSideSpec; this.rightInputSideSpec = rightInputSideSpec; -this.stateRetentionTime = stateRetentionTime; +this.leftStateRetentionTime = leftStateRetentionTime; +this.rightStateRetentionTime = rightStateRetentionTime; this.filterNullKeys = filterNullKeys; } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/StreamingJoinOperator.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/StreamingJoinOperator.java index d221c555996..308b98e2794 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/StreamingJoinOperator.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join
[flink] branch master updated (f0d01903aaa -> 62b11e2e117)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from f0d01903aaa [FLINK-28016][ci] Update CI to use Maven 3.8.6 add 62b11e2e117 [FLINK-31950][table-planner] Introduce StateMetadata No new revisions were added by this update. Summary of changes: .../planner/plan/nodes/exec/ExecNodeConfig.java| 6 + .../planner/plan/nodes/exec/StateMetadata.java | 207 ++ .../plan/nodes/exec/serde/StateMetadataTest.java | 298 + 3 files changed, 511 insertions(+) create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/StateMetadata.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/StateMetadataTest.java
[flink] branch master updated (63443aec09e -> ed9ee279e50)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 63443aec09e [FLINK-32027][runtime] Fix the potential concurrent reading bug of index file for SortMergeShuffle. new 7a423666d0f [FLINK-30815][tests] Migrate BatchAbstractTestBase to junit5 new ed9ee279e50 [FLINK-30815][tests] Migrate BatchTestBase to junit5 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: .../connectors/hive/HiveTableSourceITCase.java | 2 +- .../formats/json/JsonBatchFileSystemITCase.java| 2 +- .../batch/sql/CompactManagedTableITCase.java | 2 +- .../planner/runtime/batch/sql/FunctionITCase.java | 8 +++- .../runtime/utils/BatchAbstractTestBase.java | 38 +++- .../batch/sql/LegacyTableSourceITCase.scala| 8 ++-- .../batch/sql/PartitionableSourceITCase.scala | 7 +-- .../runtime/batch/sql/TableSinkITCase.scala| 14 +++--- .../runtime/batch/sql/TableSourceITCase.scala | 8 ++-- .../runtime/batch/sql/TimestampITCase.scala| 3 +- .../planner/runtime/utils/BatchTestBase.scala | 50 +- 11 files changed, 88 insertions(+), 54 deletions(-)
[flink] 01/02: [FLINK-30815][tests] Migrate BatchAbstractTestBase to junit5
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 7a423666d0f8452382ad5fe2635de5ad1475dd46 Author: Yuxin Tan AuthorDate: Wed Apr 19 14:27:00 2023 +0800 [FLINK-30815][tests] Migrate BatchAbstractTestBase to junit5 This closes #22427 --- .../connectors/hive/HiveTableSourceITCase.java | 2 +- .../formats/json/JsonBatchFileSystemITCase.java| 2 +- .../batch/sql/CompactManagedTableITCase.java | 2 +- .../planner/runtime/batch/sql/FunctionITCase.java | 8 +++-- .../runtime/utils/BatchAbstractTestBase.java | 38 -- .../batch/sql/LegacyTableSourceITCase.scala| 8 ++--- .../batch/sql/PartitionableSourceITCase.scala | 7 ++-- .../runtime/batch/sql/TableSinkITCase.scala| 14 .../runtime/batch/sql/TableSourceITCase.scala | 8 ++--- 9 files changed, 57 insertions(+), 32 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceITCase.java index adbf74e1798..4df9e473538 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceITCase.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceITCase.java @@ -894,7 +894,7 @@ public class HiveTableSourceITCase extends BatchAbstractTestBase { private void testCaseInsensitive(String format) throws Exception { TableEnvironment tEnv = createTableEnvWithHiveCatalog(hiveCatalog); -String folderURI = TEMPORARY_FOLDER.newFolder().toURI().toString(); +String folderURI = createTempFolder().toURI().toString(); // Flink to write sensitive fields to parquet file tEnv.executeSql( diff --git a/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java b/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java index b2efe8c863b..ff4c77d1cf9 100644 --- a/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java +++ b/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonBatchFileSystemITCase.java @@ -102,7 +102,7 @@ public class JsonBatchFileSystemITCase extends BatchFileSystemITCaseBase { } private static File generateTestData(int numRecords) throws IOException { -File tempDir = TEMPORARY_FOLDER.newFolder(); +File tempDir = createTempFolder(); File root = new File(tempDir, "id=0"); root.mkdir(); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/CompactManagedTableITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/CompactManagedTableITCase.java index c9e9e6faeaa..e5b263652a6 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/CompactManagedTableITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/CompactManagedTableITCase.java @@ -79,7 +79,7 @@ public class CompactManagedTableITCase extends BatchTestBase { try { rootPath = new Path( -new Path(TEMPORARY_FOLDER.newFolder().getPath()), +new Path(createTempFolder().getPath()), tableIdentifier.asSummaryString()); rootPath.getFileSystem().mkdirs(rootPath); } catch (IOException e) { diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/FunctionITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/FunctionITCase.java index 7d886fe4fc6..100a4042392 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/FunctionITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/FunctionITCase.java @@ -27,6 +27,8 @@ import org.apache.flink.util.UserClassLoaderJarTestUtils; import org.junit.Before; import org.junit.Test; +import java.io.File; +import java.nio.file.Files; import java.util.Arrays; import java.util.List; import java.util.Random; @@ -48,10 +50,12 @@ public class FunctionITCase extends BatchTestBase { public void before() throws Exception { super.before(); udfClassName = GENERATED_LOWER_UDF_CLASS + random.nextInt(50); +File tmpJarDir = +new File(createTempFolder(), String.format("test-jar-%s&quo
[flink] 02/02: [FLINK-30815][tests] Migrate BatchTestBase to junit5
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit ed9ee279e50781b7bd2d85f1486721c02fc7e32b Author: Yuxin Tan AuthorDate: Wed Apr 19 14:27:31 2023 +0800 [FLINK-30815][tests] Migrate BatchTestBase to junit5 This closes #22427 --- .../runtime/batch/sql/TimestampITCase.scala| 3 +- .../planner/runtime/utils/BatchTestBase.scala | 50 +- 2 files changed, 31 insertions(+), 22 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/TimestampITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/TimestampITCase.scala index c9e100da1ce..d97c68c0211 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/TimestampITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/TimestampITCase.scala @@ -24,7 +24,7 @@ import org.apache.flink.table.planner.utils.DateTimeTestUtil._ import org.apache.flink.table.planner.utils.TestDataTypeTableSource import org.apache.flink.types.Row -import org.junit.Test +import org.junit.jupiter.api.{BeforeEach, Test} import java.sql.Timestamp import java.time.{Instant, ZoneId} @@ -33,6 +33,7 @@ import scala.collection.mutable class TimestampITCase extends BatchTestBase { + @BeforeEach override def before(): Unit = { super.before() diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTestBase.scala index 1f47933f407..c1c74c69901 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTestBase.scala @@ -51,8 +51,8 @@ import org.apache.calcite.rel.RelNode import org.apache.calcite.runtime.CalciteContextException import org.apache.calcite.sql.SqlExplainLevel import org.apache.calcite.sql.parser.SqlParseException -import org.junit.{After, Assert, Before} -import org.junit.Assert._ +import org.assertj.core.api.Assertions.fail +import org.junit.jupiter.api.{AfterEach, BeforeEach} class BatchTestBase extends BatchAbstractTestBase { @@ -73,12 +73,12 @@ class BatchTestBase extends BatchAbstractTestBase { + " column ([0-9]+) to line ([0-9]+), column ([0-9]+): (.*)") @throws(classOf[Exception]) - @Before + @BeforeEach def before(): Unit = { BatchTestBase.configForMiniCluster(tableConfig) } - @After + @AfterEach def after(): Unit = { TestValuesTableFactory.clearAllData() } @@ -146,12 +146,12 @@ class BatchTestBase extends BatchAbstractTestBase { checkFunc(result).foreach { results => val plan = explainLogical(table) -Assert.fail(s""" - |Results do not match for query: - | $sqlQuery - |$results - |Plan: - | $plan +fail(s""" +|Results do not match for query: +| $sqlQuery +|$results +|Plan: +| $plan """.stripMargin) } } @@ -162,11 +162,11 @@ class BatchTestBase extends BatchAbstractTestBase { checkFunc(result).foreach { results => val plan = explainLogical(table) -Assert.fail(s""" - |Results do not match: - |$results - |Plan: - | $plan +fail(s""" +|Results do not match: +|$results +|Plan: +| $plan """.stripMargin) } } @@ -295,9 +295,9 @@ class BatchTestBase extends BatchAbstractTestBase { checkEmpty(result).foreach { results => -Assert.fail(s""" - |Results do not match for query: - |$results +fail(s""" +|Results do not match for query: +|$results """.stripMargin) } } @@ -478,7 +478,10 @@ object BatchTestBase { } def binaryRow(types: Array[LogicalType], fields: Any*): BinaryRowData = { -assertEquals("Filed count inconsistent with type information", fields.length, types.length) +// TODO, replace the failure check with a new and simpler checking method +if (fields.length != types.length) { + fail("Filed count inconsistent w
[flink] branch master updated: [FLINK-31884][table-planner] Fix the issue that upgrade ExecNode to new version causes the old serialized plan failed to pass Json SerDe round trip
This is an automated email from the ASF dual-hosted git repository. godfrey 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 3664609c762 [FLINK-31884][table-planner] Fix the issue that upgrade ExecNode to new version causes the old serialized plan failed to pass Json SerDe round trip 3664609c762 is described below commit 3664609c7622ccae80e36e85099a1b79b5935fe9 Author: Jane Chan AuthorDate: Fri Apr 21 17:41:13 2023 +0800 [FLINK-31884][table-planner] Fix the issue that upgrade ExecNode to new version causes the old serialized plan failed to pass Json SerDe round trip This closes #22459 --- .../planner/plan/nodes/exec/ExecNodeBase.java | 2 +- .../planner/plan/utils/ExecNodeMetadataUtil.java | 4 +- .../serde/ExecNodeVersionUpgradeSerdeTest.java | 102 + 3 files changed, 104 insertions(+), 4 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java index 3669e7d482d..e760d7afce3 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java @@ -82,7 +82,7 @@ public abstract class ExecNodeBase implements ExecNode { */ @JsonProperty(value = FIELD_NAME_TYPE, access = JsonProperty.Access.READ_ONLY, index = 1) protected final ExecNodeContext getContextFromAnnotation() { -return ExecNodeContext.newContext(this.getClass()).withId(getId()); +return isCompiled ? context : ExecNodeContext.newContext(this.getClass()).withId(getId()); } @JsonProperty(value = FIELD_NAME_CONFIGURATION, access = JsonProperty.Access.WRITE_ONLY) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java index 1743d645a7b..174c169d41a 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java @@ -19,7 +19,6 @@ package org.apache.flink.table.planner.plan.utils; import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigUtils; import org.apache.flink.configuration.Configuration; @@ -194,8 +193,7 @@ public final class ExecNodeMetadataUtil { || UNSUPPORTED_JSON_SERDE_CLASSES.contains(execNode); } -@VisibleForTesting -static void addTestNode(Class> execNodeClass) { +public static void addTestNode(Class> execNodeClass) { addToLookupMap(execNodeClass); } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ExecNodeVersionUpgradeSerdeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ExecNodeVersionUpgradeSerdeTest.java new file mode 100644 index 000..f0e7e3da1da --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ExecNodeVersionUpgradeSerdeTest.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.nodes.exec.serde; + +import org.apache.flink.FlinkVersion; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.planner.delegation.PlannerBase; +import org.apache.flink.table.planner.plan.nodes.
[flink] branch master updated (3e4f653ffa0 -> 0191cf7d006)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 3e4f653ffa0 [FLINK-30765][runtime] Hardens the contract of LeaderElectionService.stop add 0191cf7d006 [FLINK-31767][table-planner] Improve the implementation for "analyze table" execution on partitioned table No new revisions were added by this update. Summary of changes: .../flink/table/api/internal/AnalyzeTableUtil.java | 100 - 1 file changed, 78 insertions(+), 22 deletions(-)
[flink] branch release-1.16 updated (5c723dd7a3d -> f0361c720cb)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git from 5c723dd7a3d [FLINK-31437][jdbc][docs] Fix wrong key 'lookup.cache.caching-missing-key' in 'jdbc.md' This closes #22174. add f0361c720cb [FLINK-31273][table-planner] Fix left join with IS_NULL filter be wrongly pushed down and get wrong join results No new revisions were added by this update. Summary of changes: .../plan/rules/logical/FlinkFilterJoinRule.java| 44 +- .../rules/logical/FlinkFilterJoinRuleTest.java | 102 + .../plan/batch/sql/join/BroadcastHashJoinTest.xml | 69 .../plan/batch/sql/join/NestedLoopJoinTest.xml | 214 +++--- .../plan/batch/sql/join/ShuffledHashJoinTest.xml | 335 +-- .../plan/batch/sql/join/SortMergeJoinTest.xml | 71 .../plan/rules/logical/FlinkFilterJoinRuleTest.xml | 456 - .../planner/plan/batch/sql/join/JoinTestBase.scala | 28 +- .../plan/batch/sql/join/NestedLoopJoinTest.scala | 10 +- .../runtime/batch/sql/join/JoinITCase.scala| 108 + .../planner/runtime/stream/sql/JoinITCase.scala| 69 11 files changed, 1295 insertions(+), 211 deletions(-)
[flink] branch release-1.17 updated: [FLINK-31273][table-planner] Fix left join with IS_NULL filter be wrongly pushed down and get wrong join results
This is an automated email from the ASF dual-hosted git repository. godfrey 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 33278628dc5 [FLINK-31273][table-planner] Fix left join with IS_NULL filter be wrongly pushed down and get wrong join results 33278628dc5 is described below commit 33278628dc599bed8944733efb9495ce77993d4b Author: zhengyunhong.zyh <337361...@qq.com> AuthorDate: Wed Mar 1 15:28:50 2023 +0800 [FLINK-31273][table-planner] Fix left join with IS_NULL filter be wrongly pushed down and get wrong join results This closes #22049 (cherry picked from commit 8990822bd77d70f3249e1220a853e16dadd8ef54) --- .../plan/rules/logical/FlinkFilterJoinRule.java| 44 +- .../rules/logical/FlinkFilterJoinRuleTest.java | 102 + .../plan/batch/sql/join/BroadcastHashJoinTest.xml | 69 .../plan/batch/sql/join/NestedLoopJoinTest.xml | 214 +++--- .../plan/batch/sql/join/ShuffledHashJoinTest.xml | 335 +-- .../plan/batch/sql/join/SortMergeJoinTest.xml | 71 .../plan/rules/logical/FlinkFilterJoinRuleTest.xml | 456 - .../planner/plan/batch/sql/join/JoinTestBase.scala | 28 +- .../plan/batch/sql/join/NestedLoopJoinTest.scala | 10 +- .../runtime/batch/sql/join/JoinITCase.scala| 108 + .../planner/runtime/stream/sql/JoinITCase.scala| 69 11 files changed, 1295 insertions(+), 211 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRule.java index 47d7a44f69e..375d2ff1814 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRule.java @@ -36,11 +36,13 @@ import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexCall; import org.apache.calcite.rex.RexInputRef; +import org.apache.calcite.rex.RexLiteral; import org.apache.calcite.rex.RexNode; import org.apache.calcite.rex.RexShuttle; import org.apache.calcite.rex.RexUtil; import org.apache.calcite.rex.RexVisitor; import org.apache.calcite.rex.RexVisitorImpl; +import org.apache.calcite.sql.SqlKind; import org.apache.calcite.tools.RelBuilder; import org.apache.calcite.util.ImmutableBitSet; import org.apache.calcite.util.ImmutableIntList; @@ -53,9 +55,11 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; import static org.apache.calcite.plan.RelOptUtil.conjunctions; @@ -80,6 +84,20 @@ public abstract class FlinkFilterJoinRule public static final FlinkJoinConditionPushRule JOIN_CONDITION_PUSH = FlinkJoinConditionPushRule.FlinkFilterJoinRuleConfig.DEFAULT.toRule(); +// For left/right join, not all filter conditions support push to another side after deduction. +// This set specifies the supported filter conditions. +public static final Set SUITABLE_FILTER_TO_PUSH = +new HashSet() { +{ +add(SqlKind.EQUALS); +add(SqlKind.GREATER_THAN); +add(SqlKind.GREATER_THAN_OR_EQUAL); +add(SqlKind.LESS_THAN); +add(SqlKind.LESS_THAN_OR_EQUAL); +add(SqlKind.NOT_EQUALS); +} +}; + /** Creates a FilterJoinRule. */ protected FlinkFilterJoinRule(C config) { super(config); @@ -353,7 +371,7 @@ public abstract class FlinkFilterJoinRule for (RexNode filter : filtersToPush) { final RelOptUtil.InputFinder inputFinder = RelOptUtil.InputFinder.analyze(filter); final ImmutableBitSet inputBits = inputFinder.build(); -if (filter.isAlwaysTrue()) { +if (!isSuitableFilterToPush(filter, joinType)) { continue; } @@ -386,6 +404,30 @@ public abstract class FlinkFilterJoinRule } } +private boolean isSuitableFilterToPush(RexNode filter, JoinRelType joinType) { +if (filter.isAlwaysTrue()) { +return false; +} +if (joinType == JoinRelType.INNER) { +return true; +} +// For left/right outer join, now, we only support to push special condition in set +// SUITABLE_FILTER_TO_PUSH to other side. Take left outer join and IS_NULL con
[flink] branch master updated (cbfeef64e9b -> 8990822bd77)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from cbfeef64e9b [FLINK-31414][checkpointing] make BarrierAlignmentUtil.DelayableTimer not hidden exception add 8990822bd77 [FLINK-31273][table-planner] Fix left join with IS_NULL filter be wrongly pushed down and get wrong join results No new revisions were added by this update. Summary of changes: .../plan/rules/logical/FlinkFilterJoinRule.java| 44 +- .../rules/logical/FlinkFilterJoinRuleTest.java | 102 + .../plan/batch/sql/join/BroadcastHashJoinTest.xml | 69 .../plan/batch/sql/join/NestedLoopJoinTest.xml | 214 +++--- .../plan/batch/sql/join/ShuffledHashJoinTest.xml | 335 +-- .../plan/batch/sql/join/SortMergeJoinTest.xml | 71 .../plan/rules/logical/FlinkFilterJoinRuleTest.xml | 456 - .../planner/plan/batch/sql/join/JoinTestBase.scala | 28 +- .../plan/batch/sql/join/NestedLoopJoinTest.scala | 10 +- .../runtime/batch/sql/join/JoinITCase.scala| 108 + .../planner/runtime/stream/sql/JoinITCase.scala| 69 11 files changed, 1295 insertions(+), 211 deletions(-)
[flink] branch release-1.17 updated (7040af5b793 -> ac2eb5b977d)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch release-1.17 in repository https://gitbox.apache.org/repos/asf/flink.git from 7040af5b793 [FLINK-31185][python] Support side-output in broadcast processing new 0bdb51efd3f [FLINK-30824][hive] Add document for option 'table.exec.hive.native-agg-function.enabled' new 3bdb50513dd [FLINK-31239][hive] Fix native sum function can't get the corrected value when the argument type is string new ac2eb5b977d [FLINK-31239][hive] Add some notes in document about table.exec.hive.native-agg-function.enabled option can't turned on per job when using it via SqlClient 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: .../docs/connectors/table/hive/hive_functions.md | 29 ++ .../docs/connectors/table/hive/hive_functions.md | 29 ++ .../table/functions/hive/HiveSumAggFunction.java | 55 +- .../connectors/hive/HiveDialectAggITCase.java | 66 +- .../resources/explain/testSumAggFunctionPlan.out | 8 +-- .../planner/expressions/ExpressionBuilder.java | 10 6 files changed, 164 insertions(+), 33 deletions(-)
[flink] 03/03: [FLINK-31239][hive] Add some notes in document about table.exec.hive.native-agg-function.enabled option can't turned on per job when using it via SqlClient
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch release-1.17 in repository https://gitbox.apache.org/repos/asf/flink.git commit ac2eb5b977de47fc5550d2ee9f30fff4dcaca2b6 Author: fengli AuthorDate: Mon Feb 27 17:44:46 2023 +0800 [FLINK-31239][hive] Add some notes in document about table.exec.hive.native-agg-function.enabled option can't turned on per job when using it via SqlClient This closes #22031 (cherry picked from commit 62a3b99d23229b39c798a0b657cb11218a5bc940) --- docs/content.zh/docs/connectors/table/hive/hive_functions.md | 1 + docs/content/docs/connectors/table/hive/hive_functions.md| 1 + 2 files changed, 2 insertions(+) diff --git a/docs/content.zh/docs/connectors/table/hive/hive_functions.md b/docs/content.zh/docs/connectors/table/hive/hive_functions.md index b76331944c1..83d2955fd67 100644 --- a/docs/content.zh/docs/connectors/table/hive/hive_functions.md +++ b/docs/content.zh/docs/connectors/table/hive/hive_functions.md @@ -100,6 +100,7 @@ the option `table.exec.hive.native-agg-function.enabled`, which brings significa Attention The ability of the native aggregation functions doesn't fully align with Hive built-in aggregation functions now, for example, some data types are not supported. If performance is not a bottleneck, you don't need to turn on this option. +In addition, `table.exec.hive.native-agg-function.enabled` option can't be turned on per job when using it via SqlClient, currently, only the module level is supported. Users should turn on this option first and then load HiveModule. This issue will be fixed in the future. ## Hive User Defined Functions diff --git a/docs/content/docs/connectors/table/hive/hive_functions.md b/docs/content/docs/connectors/table/hive/hive_functions.md index 5cd7950a334..06392ea2539 100644 --- a/docs/content/docs/connectors/table/hive/hive_functions.md +++ b/docs/content/docs/connectors/table/hive/hive_functions.md @@ -100,6 +100,7 @@ the option `table.exec.hive.native-agg-function.enabled`, which brings significa Attention The ability of the native aggregation functions doesn't fully align with Hive built-in aggregation functions now, for example, some data types are not supported. If performance is not a bottleneck, you don't need to turn on this option. +In addition, `table.exec.hive.native-agg-function.enabled` option can't be turned on per job when using it via SqlClient, currently, only the module level is supported. Users should turn on this option first and then load HiveModule. This issue will be fixed in the future. ## Hive User Defined Functions
[flink] 02/03: [FLINK-31239][hive] Fix native sum function can't get the corrected value when the argument type is string
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch release-1.17 in repository https://gitbox.apache.org/repos/asf/flink.git commit 3bdb50513ddbbf6c67560a078da3f9506e5cd611 Author: fengli AuthorDate: Mon Feb 27 17:15:47 2023 +0800 [FLINK-31239][hive] Fix native sum function can't get the corrected value when the argument type is string This closes #22031 (cherry picked from commit 263555c9adcca0abe194e9a6c1d85ec591c304e4) --- .../table/functions/hive/HiveSumAggFunction.java | 55 +- .../connectors/hive/HiveDialectAggITCase.java | 66 +- .../resources/explain/testSumAggFunctionPlan.out | 8 +-- .../planner/expressions/ExpressionBuilder.java | 10 4 files changed, 106 insertions(+), 33 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveSumAggFunction.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveSumAggFunction.java index 610a1d93239..48470f997df 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveSumAggFunction.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveSumAggFunction.java @@ -22,13 +22,20 @@ import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableException; import org.apache.flink.table.expressions.Expression; import org.apache.flink.table.expressions.UnresolvedReferenceExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.inference.CallContext; +import java.math.BigDecimal; + import static org.apache.flink.connectors.hive.HiveOptions.TABLE_EXEC_HIVE_NATIVE_AGG_FUNCTION_ENABLED; import static org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef; +import static org.apache.flink.table.expressions.ApiExpressionUtils.valueLiteral; +import static org.apache.flink.table.planner.expressions.ExpressionBuilder.and; +import static org.apache.flink.table.planner.expressions.ExpressionBuilder.coalesce; import static org.apache.flink.table.planner.expressions.ExpressionBuilder.ifThenElse; import static org.apache.flink.table.planner.expressions.ExpressionBuilder.isNull; +import static org.apache.flink.table.planner.expressions.ExpressionBuilder.isTrue; import static org.apache.flink.table.planner.expressions.ExpressionBuilder.nullOf; import static org.apache.flink.table.planner.expressions.ExpressionBuilder.tryCast; import static org.apache.flink.table.planner.expressions.ExpressionBuilder.typeLiteral; @@ -40,7 +47,10 @@ import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getSc public class HiveSumAggFunction extends HiveDeclarativeAggregateFunction { private final UnresolvedReferenceExpression sum = unresolvedRef("sum"); +private final UnresolvedReferenceExpression isEmpty = unresolvedRef("isEmpty"); + private DataType resultType; +private ValueLiteralExpression zero; @Override public int operandCount() { @@ -49,12 +59,12 @@ public class HiveSumAggFunction extends HiveDeclarativeAggregateFunction { @Override public UnresolvedReferenceExpression[] aggBufferAttributes() { -return new UnresolvedReferenceExpression[] {sum}; +return new UnresolvedReferenceExpression[] {sum, isEmpty}; } @Override public DataType[] getAggBufferTypes() { -return new DataType[] {getResultType()}; +return new DataType[] {getResultType(), DataTypes.BOOLEAN()}; } @Override @@ -64,20 +74,19 @@ public class HiveSumAggFunction extends HiveDeclarativeAggregateFunction { @Override public Expression[] initialValuesExpressions() { -return new Expression[] {/* sum = */ nullOf(getResultType())}; +return new Expression[] {/* sum = */ nullOf(getResultType()), valueLiteral(true)}; } @Override public Expression[] accumulateExpressions() { Expression tryCastOperand = tryCast(operand(0), typeLiteral(getResultType())); +Expression coalesceSum = coalesce(sum, zero); return new Expression[] { /* sum = */ ifThenElse( isNull(tryCastOperand), -sum, -ifThenElse( -isNull(sum), -tryCastOperand, -adjustedPlus(getResultType(), sum, tryCastOperand))) +coalesceSum, +adjustedPlus(getResultType(), coalesceSum, tryCastOperand)), +and(isEmpty, isNull(operand(0))) }; } @@ -88,20 +97,19 @@ public class HiveSumAggFunction extends HiveDeclarativeAggregateFunction { @Override
[flink] 01/03: [FLINK-30824][hive] Add document for option 'table.exec.hive.native-agg-function.enabled'
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch release-1.17 in repository https://gitbox.apache.org/repos/asf/flink.git commit 0bdb51efd3f4b3050545e5cfbef9f587d657f9be Author: Ron AuthorDate: Mon Feb 20 20:52:47 2023 +0800 [FLINK-30824][hive] Add document for option 'table.exec.hive.native-agg-function.enabled' This closes #21789 (cherry picked from commit afdc079465c393d98bf2b3607a75b1fc9d58d281) --- .../docs/connectors/table/hive/hive_functions.md | 28 ++ .../docs/connectors/table/hive/hive_functions.md | 28 ++ 2 files changed, 56 insertions(+) diff --git a/docs/content.zh/docs/connectors/table/hive/hive_functions.md b/docs/content.zh/docs/connectors/table/hive/hive_functions.md index da540f9399a..b76331944c1 100644 --- a/docs/content.zh/docs/connectors/table/hive/hive_functions.md +++ b/docs/content.zh/docs/connectors/table/hive/hive_functions.md @@ -73,6 +73,34 @@ Some Hive built-in functions in older versions have [thread safety issues](https We recommend users patch their own Hive to fix them. {{< /hint >}} +## Use Native Hive Aggregate Functions + +If [HiveModule]({{< ref "docs/dev/table/modules" >}}#hivemodule) is loaded with a higher priority than CoreModule, Flink will try to use the Hive built-in function first. And then for Hive built-in aggregation functions, +Flink can only use the sort-based aggregation operator now. From Flink 1.17, we have introduced some native hive aggregation functions, which can be executed using the hash-based aggregation operator. +Currently, only five functions are supported, namely sum/count/avg/min/max, and more aggregation functions will be supported in the future. Users can use the native aggregation function by turning on +the option `table.exec.hive.native-agg-function.enabled`, which brings significant performance improvement to the job. + + + + +Key +Default +Type +Description + + + + +table.exec.hive.native-agg-function.enabled +false +Boolean +Enabling to use native aggregation functions, hash-based aggregation strategy could be used that can improve the aggregation performance. This is a job-level option. + + + + +Attention The ability of the native aggregation functions doesn't fully align with Hive built-in aggregation functions now, for example, some data types are not supported. If performance is not a bottleneck, you don't need to turn on this option. + ## Hive User Defined Functions Users can use their existing Hive User Defined Functions in Flink. diff --git a/docs/content/docs/connectors/table/hive/hive_functions.md b/docs/content/docs/connectors/table/hive/hive_functions.md index e57d27f1804..5cd7950a334 100644 --- a/docs/content/docs/connectors/table/hive/hive_functions.md +++ b/docs/content/docs/connectors/table/hive/hive_functions.md @@ -73,6 +73,34 @@ Some Hive built-in functions in older versions have [thread safety issues](https We recommend users patch their own Hive to fix them. {{< /hint >}} +## Use Native Hive Aggregate Functions + +If [HiveModule]({{< ref "docs/dev/table/modules" >}}#hivemodule) is loaded with a higher priority than CoreModule, Flink will try to use the Hive built-in function first. And then for Hive built-in aggregation functions, +Flink can only use the sort-based aggregation operator now. From Flink 1.17, we have introduced some native hive aggregation functions, which can be executed using the hash-based aggregation operator. +Currently, only five functions are supported, namely sum/count/avg/min/max, and more aggregation functions will be supported in the future. Users can use the native aggregation function by turning on +the option `table.exec.hive.native-agg-function.enabled`, which brings significant performance improvement to the job. + + + + +Key +Default +Type +Description + + + + +table.exec.hive.native-agg-function.enabled +false +Boolean +Enabling to use native aggregation functions, hash-based aggregation strategy could be used that can improve the aggregation performance. This is a job-level option. + + + + +Attention The ability of the native aggregation functions doesn't fully align with Hive built-in aggregation functions now, for example, some data types are not supported. If performance is not a bottleneck, you don't need to turn on this option. + ## Hive User Defined Functions Users can use their existing Hive User Defined Functions in Flink.
[flink] 01/02: [FLINK-31239][hive] Fix native sum function can't get the corrected value when the argument type is string
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 263555c9adcca0abe194e9a6c1d85ec591c304e4 Author: fengli AuthorDate: Mon Feb 27 17:15:47 2023 +0800 [FLINK-31239][hive] Fix native sum function can't get the corrected value when the argument type is string This closes #22031 --- .../table/functions/hive/HiveSumAggFunction.java | 55 +- .../connectors/hive/HiveDialectAggITCase.java | 66 +- .../resources/explain/testSumAggFunctionPlan.out | 8 +-- .../planner/expressions/ExpressionBuilder.java | 10 4 files changed, 106 insertions(+), 33 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveSumAggFunction.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveSumAggFunction.java index 610a1d93239..48470f997df 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveSumAggFunction.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveSumAggFunction.java @@ -22,13 +22,20 @@ import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableException; import org.apache.flink.table.expressions.Expression; import org.apache.flink.table.expressions.UnresolvedReferenceExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.inference.CallContext; +import java.math.BigDecimal; + import static org.apache.flink.connectors.hive.HiveOptions.TABLE_EXEC_HIVE_NATIVE_AGG_FUNCTION_ENABLED; import static org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef; +import static org.apache.flink.table.expressions.ApiExpressionUtils.valueLiteral; +import static org.apache.flink.table.planner.expressions.ExpressionBuilder.and; +import static org.apache.flink.table.planner.expressions.ExpressionBuilder.coalesce; import static org.apache.flink.table.planner.expressions.ExpressionBuilder.ifThenElse; import static org.apache.flink.table.planner.expressions.ExpressionBuilder.isNull; +import static org.apache.flink.table.planner.expressions.ExpressionBuilder.isTrue; import static org.apache.flink.table.planner.expressions.ExpressionBuilder.nullOf; import static org.apache.flink.table.planner.expressions.ExpressionBuilder.tryCast; import static org.apache.flink.table.planner.expressions.ExpressionBuilder.typeLiteral; @@ -40,7 +47,10 @@ import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getSc public class HiveSumAggFunction extends HiveDeclarativeAggregateFunction { private final UnresolvedReferenceExpression sum = unresolvedRef("sum"); +private final UnresolvedReferenceExpression isEmpty = unresolvedRef("isEmpty"); + private DataType resultType; +private ValueLiteralExpression zero; @Override public int operandCount() { @@ -49,12 +59,12 @@ public class HiveSumAggFunction extends HiveDeclarativeAggregateFunction { @Override public UnresolvedReferenceExpression[] aggBufferAttributes() { -return new UnresolvedReferenceExpression[] {sum}; +return new UnresolvedReferenceExpression[] {sum, isEmpty}; } @Override public DataType[] getAggBufferTypes() { -return new DataType[] {getResultType()}; +return new DataType[] {getResultType(), DataTypes.BOOLEAN()}; } @Override @@ -64,20 +74,19 @@ public class HiveSumAggFunction extends HiveDeclarativeAggregateFunction { @Override public Expression[] initialValuesExpressions() { -return new Expression[] {/* sum = */ nullOf(getResultType())}; +return new Expression[] {/* sum = */ nullOf(getResultType()), valueLiteral(true)}; } @Override public Expression[] accumulateExpressions() { Expression tryCastOperand = tryCast(operand(0), typeLiteral(getResultType())); +Expression coalesceSum = coalesce(sum, zero); return new Expression[] { /* sum = */ ifThenElse( isNull(tryCastOperand), -sum, -ifThenElse( -isNull(sum), -tryCastOperand, -adjustedPlus(getResultType(), sum, tryCastOperand))) +coalesceSum, +adjustedPlus(getResultType(), coalesceSum, tryCastOperand)), +and(isEmpty, isNull(operand(0))) }; } @@ -88,20 +97,19 @@ public class HiveSumAggFunction extends HiveDeclarativeAggregateFunction { @Override public Expression[] mergeExpressions() { +Expression coalesceSum
[flink] 02/02: [FLINK-31239][hive] Add some notes in document about table.exec.hive.native-agg-function.enabled option can't turned on per job when using it via SqlClient
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 62a3b99d23229b39c798a0b657cb11218a5bc940 Author: fengli AuthorDate: Mon Feb 27 17:44:46 2023 +0800 [FLINK-31239][hive] Add some notes in document about table.exec.hive.native-agg-function.enabled option can't turned on per job when using it via SqlClient This closes #22031 --- docs/content.zh/docs/connectors/table/hive/hive_functions.md | 1 + docs/content/docs/connectors/table/hive/hive_functions.md| 1 + 2 files changed, 2 insertions(+) diff --git a/docs/content.zh/docs/connectors/table/hive/hive_functions.md b/docs/content.zh/docs/connectors/table/hive/hive_functions.md index b76331944c1..83d2955fd67 100644 --- a/docs/content.zh/docs/connectors/table/hive/hive_functions.md +++ b/docs/content.zh/docs/connectors/table/hive/hive_functions.md @@ -100,6 +100,7 @@ the option `table.exec.hive.native-agg-function.enabled`, which brings significa Attention The ability of the native aggregation functions doesn't fully align with Hive built-in aggregation functions now, for example, some data types are not supported. If performance is not a bottleneck, you don't need to turn on this option. +In addition, `table.exec.hive.native-agg-function.enabled` option can't be turned on per job when using it via SqlClient, currently, only the module level is supported. Users should turn on this option first and then load HiveModule. This issue will be fixed in the future. ## Hive User Defined Functions diff --git a/docs/content/docs/connectors/table/hive/hive_functions.md b/docs/content/docs/connectors/table/hive/hive_functions.md index 5cd7950a334..06392ea2539 100644 --- a/docs/content/docs/connectors/table/hive/hive_functions.md +++ b/docs/content/docs/connectors/table/hive/hive_functions.md @@ -100,6 +100,7 @@ the option `table.exec.hive.native-agg-function.enabled`, which brings significa Attention The ability of the native aggregation functions doesn't fully align with Hive built-in aggregation functions now, for example, some data types are not supported. If performance is not a bottleneck, you don't need to turn on this option. +In addition, `table.exec.hive.native-agg-function.enabled` option can't be turned on per job when using it via SqlClient, currently, only the module level is supported. Users should turn on this option first and then load HiveModule. This issue will be fixed in the future. ## Hive User Defined Functions
[flink] branch master updated (d96bb2f66d7 -> 62a3b99d232)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from d96bb2f66d7 [FLINK-30948][sql-client] Fix ExecutorImpl#testInterruptException hangs (#22055) new 263555c9adc [FLINK-31239][hive] Fix native sum function can't get the corrected value when the argument type is string new 62a3b99d232 [FLINK-31239][hive] Add some notes in document about table.exec.hive.native-agg-function.enabled option can't turned on per job when using it via SqlClient 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/connectors/table/hive/hive_functions.md | 1 + .../docs/connectors/table/hive/hive_functions.md | 1 + .../table/functions/hive/HiveSumAggFunction.java | 55 +- .../connectors/hive/HiveDialectAggITCase.java | 66 +- .../resources/explain/testSumAggFunctionPlan.out | 8 +-- .../planner/expressions/ExpressionBuilder.java | 10 6 files changed, 108 insertions(+), 33 deletions(-)
[flink] branch release-1.17 updated: [FLINK-30989][table] Fix some options don't take effect in batch mode
This is an automated email from the ASF dual-hosted git repository. godfrey 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 33308811399 [FLINK-30989][table] Fix some options don't take effect in batch mode 33308811399 is described below commit 333088113993f4607038dae391863b5c30d0bc95 Author: fengli AuthorDate: Sun Feb 26 15:59:33 2023 +0800 [FLINK-30989][table] Fix some options don't take effect in batch mode This closes #22024 (cherry picked from commit b4d43b47c993b7b4d5e4f7a78610c54124fcbcb4) --- .../nodes/exec/batch/BatchExecHashAggregate.java | 9 +++- .../plan/nodes/exec/batch/BatchExecHashJoin.java | 10 .../plan/nodes/exec/batch/BatchExecSort.java | 10 +++- .../plan/utils/SorMergeJoinOperatorUtil.java | 16 ++ .../planner/codegen/LongHashJoinGenerator.scala| 5 +- .../codegen/agg/batch/HashAggCodeGenHelper.scala | 17 -- .../codegen/agg/batch/HashAggCodeGenerator.scala | 11 ++-- .../codegen/LongAdaptiveHashJoinGeneratorTest.java | 6 +++ .../agg/batch/HashAggCodeGeneratorTest.scala | 7 ++- .../runtime/hashtable/BaseHybridHashTable.java | 23 +++- .../table/runtime/hashtable/BinaryHashTable.java | 11 ++-- .../table/runtime/hashtable/LongHashPartition.java | 4 +- .../runtime/hashtable/LongHybridHashTable.java | 13 ++--- .../runtime/operators/join/HashJoinOperator.java | 13 - .../operators/join/SortMergeJoinFunction.java | 23 ++-- .../sort/AbstractBinaryExternalMerger.java | 10 ++-- .../operators/sort/BinaryExternalMerger.java | 4 +- .../operators/sort/BinaryExternalSorter.java | 46 .../operators/sort/BinaryInMemorySortBuffer.java | 4 +- .../operators/sort/BinaryKVExternalMerger.java | 4 +- .../operators/sort/BinaryKVInMemorySortBuffer.java | 3 +- .../operators/sort/BufferedKVExternalSorter.java | 28 -- .../table/runtime/operators/sort/SortOperator.java | 21 +++- .../flink/table/runtime/util/FileChannelUtil.java | 8 +-- .../runtime/hashtable/BinaryHashTableTest.java | 25 - .../table/runtime/hashtable/LongHashTableTest.java | 10 ++-- .../runtime/operators/aggregate/HashAggTest.java | 6 --- .../aggregate/SumHashAggTestOperator.java | 16 +++--- .../join/Int2HashJoinOperatorTestBase.java | 23 .../join/Int2SortMergeJoinOperatorTest.java| 16 ++ .../join/String2HashJoinOperatorTest.java | 19 +++ .../join/String2SortMergeJoinOperatorTest.java | 16 ++ .../operators/sort/BinaryExternalSorterTest.java | 63 +++--- .../sort/BufferedKVExternalSorterTest.java | 8 ++- 34 files changed, 367 insertions(+), 141 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java index 27d672706ed..49ea262626a 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java @@ -139,7 +139,14 @@ public class BatchExecHashAggregate extends ExecNodeBase auxGrouping, isMerge, isFinal, -supportAdaptiveLocalHashAgg); +supportAdaptiveLocalHashAgg, + config.get(ExecutionConfigOptions.TABLE_EXEC_SORT_MAX_NUM_FILE_HANDLES), + config.get(ExecutionConfigOptions.TABLE_EXEC_SPILL_COMPRESSION_ENABLED), +(int) +config.get( +ExecutionConfigOptions + .TABLE_EXEC_SPILL_COMPRESSION_BLOCK_SIZE) +.getBytes()); } return ExecNodeUtil.createOneInputTransformation( diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java index d390292a901..ef178487eff 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java @@ -213,6 +213,12 @@ public class BatchExecHashJoin
[flink] branch master updated: [FLINK-30989][table] Fix some options don't take effect in batch mode
This is an automated email from the ASF dual-hosted git repository. godfrey 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 b4d43b47c99 [FLINK-30989][table] Fix some options don't take effect in batch mode b4d43b47c99 is described below commit b4d43b47c993b7b4d5e4f7a78610c54124fcbcb4 Author: fengli AuthorDate: Sun Feb 26 15:59:33 2023 +0800 [FLINK-30989][table] Fix some options don't take effect in batch mode This closes #22024 --- .../nodes/exec/batch/BatchExecHashAggregate.java | 9 +++- .../plan/nodes/exec/batch/BatchExecHashJoin.java | 10 .../plan/nodes/exec/batch/BatchExecSort.java | 10 +++- .../plan/utils/SorMergeJoinOperatorUtil.java | 16 ++ .../planner/codegen/LongHashJoinGenerator.scala| 5 +- .../codegen/agg/batch/HashAggCodeGenHelper.scala | 17 -- .../codegen/agg/batch/HashAggCodeGenerator.scala | 11 ++-- .../codegen/LongAdaptiveHashJoinGeneratorTest.java | 6 +++ .../agg/batch/HashAggCodeGeneratorTest.scala | 7 ++- .../runtime/hashtable/BaseHybridHashTable.java | 23 +++- .../table/runtime/hashtable/BinaryHashTable.java | 11 ++-- .../table/runtime/hashtable/LongHashPartition.java | 4 +- .../runtime/hashtable/LongHybridHashTable.java | 13 ++--- .../runtime/operators/join/HashJoinOperator.java | 13 - .../operators/join/SortMergeJoinFunction.java | 23 ++-- .../sort/AbstractBinaryExternalMerger.java | 10 ++-- .../operators/sort/BinaryExternalMerger.java | 4 +- .../operators/sort/BinaryExternalSorter.java | 46 .../operators/sort/BinaryInMemorySortBuffer.java | 4 +- .../operators/sort/BinaryKVExternalMerger.java | 4 +- .../operators/sort/BinaryKVInMemorySortBuffer.java | 3 +- .../operators/sort/BufferedKVExternalSorter.java | 28 -- .../table/runtime/operators/sort/SortOperator.java | 21 +++- .../flink/table/runtime/util/FileChannelUtil.java | 8 +-- .../runtime/hashtable/BinaryHashTableTest.java | 25 - .../table/runtime/hashtable/LongHashTableTest.java | 10 ++-- .../runtime/operators/aggregate/HashAggTest.java | 6 --- .../aggregate/SumHashAggTestOperator.java | 16 +++--- .../join/Int2HashJoinOperatorTestBase.java | 23 .../join/Int2SortMergeJoinOperatorTest.java| 16 ++ .../join/String2HashJoinOperatorTest.java | 19 +++ .../join/String2SortMergeJoinOperatorTest.java | 16 ++ .../operators/sort/BinaryExternalSorterTest.java | 63 +++--- .../sort/BufferedKVExternalSorterTest.java | 8 ++- 34 files changed, 367 insertions(+), 141 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java index 27d672706ed..49ea262626a 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java @@ -139,7 +139,14 @@ public class BatchExecHashAggregate extends ExecNodeBase auxGrouping, isMerge, isFinal, -supportAdaptiveLocalHashAgg); +supportAdaptiveLocalHashAgg, + config.get(ExecutionConfigOptions.TABLE_EXEC_SORT_MAX_NUM_FILE_HANDLES), + config.get(ExecutionConfigOptions.TABLE_EXEC_SPILL_COMPRESSION_ENABLED), +(int) +config.get( +ExecutionConfigOptions + .TABLE_EXEC_SPILL_COMPRESSION_BLOCK_SIZE) +.getBytes()); } return ExecNodeUtil.createOneInputTransformation( diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java index d390292a901..ef178487eff 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java @@ -213,6 +213,12 @@ public class BatchExecHashJoin extends ExecNodeBase condFunc, 1.0
[flink] branch release-1.16 updated (5b9398ae067 -> 247a099cc35)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git from 5b9398ae067 [FLINK-29859][e2e] Running TPC-DS with adaptive batch scheduler supports custom errors check. add 247a099cc35 [FLINK-31082][table-planner] Setting maven property 'flink.resueForks' to false in table planner module No new revisions were added by this update. Summary of changes: flink-table/flink-table-planner/pom.xml | 7 +-- 1 file changed, 5 insertions(+), 2 deletions(-)
[flink] branch release-1.17 updated: [FLINK-30971][table-planner] Modify the default value of parameter 'table.exec.local-hash-agg.adaptive.sampling-threshold'
This is an automated email from the ASF dual-hosted git repository. godfrey 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 74743d04bf1 [FLINK-30971][table-planner] Modify the default value of parameter 'table.exec.local-hash-agg.adaptive.sampling-threshold' 74743d04bf1 is described below commit 74743d04bf100fc1202a320db02a7d6964266858 Author: yunhong <337361...@qq.com> AuthorDate: Fri Feb 17 15:10:45 2023 +0800 [FLINK-30971][table-planner] Modify the default value of parameter 'table.exec.local-hash-agg.adaptive.sampling-threshold' This closes #21900 (cherry picked from commit 55b927b0e6eb2d5d71487b9bb2c4dab80017a7e5) --- .../flink/table/planner/codegen/agg/batch/HashAggCodeGenerator.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/HashAggCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/HashAggCodeGenerator.scala index e98ca5dd266..d13fd831522 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/HashAggCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/HashAggCodeGenerator.scala @@ -67,7 +67,7 @@ object HashAggCodeGenerator { val TABLE_EXEC_LOCAL_HASH_AGG_ADAPTIVE_SAMPLING_THRESHOLD: ConfigOption[JLong] = key("table.exec.local-hash-agg.adaptive.sampling-threshold") .longType() - .defaultValue(Long.box(500L)) + .defaultValue(Long.box(50L)) .withDescription( s""" |If adaptive local hash aggregation is enabled, this value defines how @@ -76,7 +76,7 @@ object HashAggCodeGenerator { |for the local aggregate. The higher the sampling threshold, the more accurate |the distinct value rate is. But as the sampling threshold increases, local |aggregation is meaningless when the distinct values rate is low. - |The default value is 500. + |The default value is 50. |""".stripMargin) @Experimental
[flink] branch master updated: [FLINK-30824][hive] Add document for option 'table.exec.hive.native-agg-function.enabled'
This is an automated email from the ASF dual-hosted git repository. godfrey 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 afdc079465c [FLINK-30824][hive] Add document for option 'table.exec.hive.native-agg-function.enabled' afdc079465c is described below commit afdc079465c393d98bf2b3607a75b1fc9d58d281 Author: Ron AuthorDate: Mon Feb 20 20:52:47 2023 +0800 [FLINK-30824][hive] Add document for option 'table.exec.hive.native-agg-function.enabled' This closes #21789 --- .../docs/connectors/table/hive/hive_functions.md | 28 ++ .../docs/connectors/table/hive/hive_functions.md | 28 ++ 2 files changed, 56 insertions(+) diff --git a/docs/content.zh/docs/connectors/table/hive/hive_functions.md b/docs/content.zh/docs/connectors/table/hive/hive_functions.md index da540f9399a..b76331944c1 100644 --- a/docs/content.zh/docs/connectors/table/hive/hive_functions.md +++ b/docs/content.zh/docs/connectors/table/hive/hive_functions.md @@ -73,6 +73,34 @@ Some Hive built-in functions in older versions have [thread safety issues](https We recommend users patch their own Hive to fix them. {{< /hint >}} +## Use Native Hive Aggregate Functions + +If [HiveModule]({{< ref "docs/dev/table/modules" >}}#hivemodule) is loaded with a higher priority than CoreModule, Flink will try to use the Hive built-in function first. And then for Hive built-in aggregation functions, +Flink can only use the sort-based aggregation operator now. From Flink 1.17, we have introduced some native hive aggregation functions, which can be executed using the hash-based aggregation operator. +Currently, only five functions are supported, namely sum/count/avg/min/max, and more aggregation functions will be supported in the future. Users can use the native aggregation function by turning on +the option `table.exec.hive.native-agg-function.enabled`, which brings significant performance improvement to the job. + + + + +Key +Default +Type +Description + + + + +table.exec.hive.native-agg-function.enabled +false +Boolean +Enabling to use native aggregation functions, hash-based aggregation strategy could be used that can improve the aggregation performance. This is a job-level option. + + + + +Attention The ability of the native aggregation functions doesn't fully align with Hive built-in aggregation functions now, for example, some data types are not supported. If performance is not a bottleneck, you don't need to turn on this option. + ## Hive User Defined Functions Users can use their existing Hive User Defined Functions in Flink. diff --git a/docs/content/docs/connectors/table/hive/hive_functions.md b/docs/content/docs/connectors/table/hive/hive_functions.md index e57d27f1804..5cd7950a334 100644 --- a/docs/content/docs/connectors/table/hive/hive_functions.md +++ b/docs/content/docs/connectors/table/hive/hive_functions.md @@ -73,6 +73,34 @@ Some Hive built-in functions in older versions have [thread safety issues](https We recommend users patch their own Hive to fix them. {{< /hint >}} +## Use Native Hive Aggregate Functions + +If [HiveModule]({{< ref "docs/dev/table/modules" >}}#hivemodule) is loaded with a higher priority than CoreModule, Flink will try to use the Hive built-in function first. And then for Hive built-in aggregation functions, +Flink can only use the sort-based aggregation operator now. From Flink 1.17, we have introduced some native hive aggregation functions, which can be executed using the hash-based aggregation operator. +Currently, only five functions are supported, namely sum/count/avg/min/max, and more aggregation functions will be supported in the future. Users can use the native aggregation function by turning on +the option `table.exec.hive.native-agg-function.enabled`, which brings significant performance improvement to the job. + + + + +Key +Default +Type +Description + + + + +table.exec.hive.native-agg-function.enabled +false +Boolean +Enabling to use native aggregation functions, hash-based aggregation strategy could be used that can improve the aggregation performance. This is a job-level option. + + + + +Attention The ability of the native aggregation functions doesn't fully align with Hive built-in aggregation functions now, for example, some data types are not supported. If performance is not a bottleneck, you don't need to turn on this option. + ## Hive User Defined Functions Users can use their existing Hive User Defined Functions in Flink.
[flink] branch master updated: [FLINK-30971][table-planner] Modify the default value of parameter 'table.exec.local-hash-agg.adaptive.sampling-threshold'
This is an automated email from the ASF dual-hosted git repository. godfrey 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 55b927b0e6e [FLINK-30971][table-planner] Modify the default value of parameter 'table.exec.local-hash-agg.adaptive.sampling-threshold' 55b927b0e6e is described below commit 55b927b0e6eb2d5d71487b9bb2c4dab80017a7e5 Author: yunhong <337361...@qq.com> AuthorDate: Fri Feb 17 15:10:45 2023 +0800 [FLINK-30971][table-planner] Modify the default value of parameter 'table.exec.local-hash-agg.adaptive.sampling-threshold' This closes #21900 --- .../flink/table/planner/codegen/agg/batch/HashAggCodeGenerator.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/HashAggCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/HashAggCodeGenerator.scala index e98ca5dd266..d13fd831522 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/HashAggCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/HashAggCodeGenerator.scala @@ -67,7 +67,7 @@ object HashAggCodeGenerator { val TABLE_EXEC_LOCAL_HASH_AGG_ADAPTIVE_SAMPLING_THRESHOLD: ConfigOption[JLong] = key("table.exec.local-hash-agg.adaptive.sampling-threshold") .longType() - .defaultValue(Long.box(500L)) + .defaultValue(Long.box(50L)) .withDescription( s""" |If adaptive local hash aggregation is enabled, this value defines how @@ -76,7 +76,7 @@ object HashAggCodeGenerator { |for the local aggregate. The higher the sampling threshold, the more accurate |the distinct value rate is. But as the sampling threshold increases, local |aggregation is meaningless when the distinct values rate is low. - |The default value is 500. + |The default value is 50. |""".stripMargin) @Experimental
[flink] branch master updated (1c0870ae087 -> 81aebd85580)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 1c0870ae087 [FLINK-31055][runtime] Fix the bug that dynamic flag of stream graph does not take effect when translating the transformations add 81aebd85580 [FLINK-31082][table-planner] Fix for setting maven property 'flink.resueForks' to false in table planner module in right way No new revisions were added by this update. Summary of changes: flink-table/flink-table-planner/pom.xml | 15 +-- 1 file changed, 5 insertions(+), 10 deletions(-)
[flink] branch release-1.16 updated (793b0f791c4 -> 00a4ab9011c)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git from 793b0f791c4 [FLINK-29405][table] Fix unstable test InputFormatCacheLoaderTest add 00a4ab9011c [FLINK-30876][table-planner] Fix ResetTransformationProcessor don't reset the transformations in BatchExecMultiInput No new revisions were added by this update. Summary of changes: .../planner/plan/nodes/exec/ExecNodeBase.java | 7 + .../nodes/exec/batch/BatchExecMultipleInput.java | 24 .../MultipleInputNodeCreationProcessorTest.java| 12 +- .../ResetTransformationProcessorTest.java | 143 + .../flink/table/planner/utils/TableTestBase.scala | 13 +- 5 files changed, 187 insertions(+), 12 deletions(-) create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/processor/ResetTransformationProcessorTest.java
[flink] branch master updated: [FLINK-30876][table-planner] Fix ResetTransformationProcessor don't reset the transformations in BatchExecMultiInput
This is an automated email from the ASF dual-hosted git repository. godfrey 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 111342f37bd [FLINK-30876][table-planner] Fix ResetTransformationProcessor don't reset the transformations in BatchExecMultiInput 111342f37bd is described below commit 111342f37bdc0d582d3f7af458d9869f0548299f Author: zhengyunhong.zyh <337361...@qq.com> AuthorDate: Thu Feb 2 16:24:50 2023 +0800 [FLINK-30876][table-planner] Fix ResetTransformationProcessor don't reset the transformations in BatchExecMultiInput This closes #21828 --- .../planner/plan/nodes/exec/ExecNodeBase.java | 7 + .../nodes/exec/batch/BatchExecMultipleInput.java | 24 .../MultipleInputNodeCreationProcessorTest.java| 12 +- .../ResetTransformationProcessorTest.java | 143 + .../flink/table/planner/utils/TableTestBase.scala | 13 +- 5 files changed, 187 insertions(+), 12 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java index f765bcbd720..3669e7d482d 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.dag.Transformation; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; @@ -276,4 +277,10 @@ public abstract class ExecNodeBase implements ExecNode { public void resetTransformation() { this.transformation = null; } + +@VisibleForTesting +@JsonIgnore +public Transformation getTransformation() { +return this.transformation; +} } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecMultipleInput.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecMultipleInput.java index 8ab69e34d18..93b5bf84937 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecMultipleInput.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecMultipleInput.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.dag.Transformation; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.ChainingStrategy; @@ -32,6 +33,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; import org.apache.flink.table.planner.plan.nodes.exec.InputProperty; import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator; import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil; +import org.apache.flink.table.planner.plan.nodes.exec.visitor.AbstractExecNodeExactlyOnceVisitor; import org.apache.flink.table.runtime.operators.multipleinput.BatchMultipleInputStreamOperatorFactory; import org.apache.flink.table.runtime.operators.multipleinput.TableOperatorWrapperGenerator; import org.apache.flink.table.runtime.operators.multipleinput.input.InputSpec; @@ -149,7 +151,29 @@ public class BatchExecMultipleInput extends ExecNodeBase return multipleInputTransform; } +@Override +public void resetTransformation() { +super.resetTransformation(); +// For BatchExecMultipleInput, we also need to reset transformation for +// rootNode in BatchExecMultipleInput. +AbstractExecNodeExactlyOnceVisitor visitor = +new AbstractExecNodeExactlyOnceVisitor() { + +@Override +protected void visitNode(ExecNode node) { +((ExecNodeBase) node).resetTransformation(); +visitInputs(node); +} +}; +rootNode.accept(visitor); +} + public List getOriginalEdges() { return originalEdges; } + +@VisibleForTesting +public ExecNode getRootNode() { +return rootNode; +} } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessorTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/pro
[flink] branch master updated (30dd8178dd0 -> 3ac2c330bad)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 30dd8178dd0 [FLINK-30855][Connector/Google Cloud PubSub] Remove GCP PubSub connector. This closes #21811 add 3ac2c330bad [hotfix][table-planner] Fix ITCase IOException due to network buffer insufficient when the default parallelism don't set manually in test No new revisions were added by this update. Summary of changes: .../planner/runtime/stream/sql/join/JoinReorderITCase.java | 10 -- .../table/planner/runtime/utils/JoinReorderITCaseBase.java | 9 + .../flink/table/planner/utils/StatisticsReportTestBase.java| 8 3 files changed, 17 insertions(+), 10 deletions(-)
[flink] branch master updated: [FLINK-30542][table-planner] Introduce adaptive local hash aggregate to adaptively determine whether local hash aggregate is required at runtime
This is an automated email from the ASF dual-hosted git repository. godfrey 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 122ba8f319b [FLINK-30542][table-planner] Introduce adaptive local hash aggregate to adaptively determine whether local hash aggregate is required at runtime 122ba8f319b is described below commit 122ba8f319b0d68374abba08d676e6dfa82cc114 Author: zhengyunhong.zyh <337361...@qq.com> AuthorDate: Wed Jan 11 21:57:02 2023 +0800 [FLINK-30542][table-planner] Introduce adaptive local hash aggregate to adaptively determine whether local hash aggregate is required at runtime This closes #21586 --- .../nodes/exec/batch/BatchExecHashAggregate.java | 25 +- .../planner/codegen/ProjectionCodeGenerator.scala | 166 ++- .../codegen/agg/batch/HashAggCodeGenerator.scala | 205 +++-- .../batch/BatchPhysicalHashAggregate.scala | 1 + .../batch/BatchPhysicalLocalHashAggregate.scala| 3 + .../physical/batch/BatchPhysicalAggRuleBase.scala | 4 +- .../physical/batch/BatchPhysicalHashAggRule.scala | 10 +- .../physical/batch/BatchPhysicalJoinRuleBase.scala | 1 + .../physical/batch/BatchPhysicalSortAggRule.scala | 4 +- .../physical/batch/EnforceLocalAggRuleBase.scala | 3 +- .../table/planner/plan/utils/AggregateUtil.scala | 20 ++ .../agg/batch/HashAggCodeGeneratorTest.scala | 8 +- .../plan/metadata/FlinkRelMdHandlerTestBase.scala | 3 + .../batch/sql/agg/AggregateITCaseBase.scala| 15 +- .../runtime/batch/sql/agg/HashAggITCase.scala | 316 - 15 files changed, 733 insertions(+), 51 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java index 02ed8a737da..53ebbf2132c 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java @@ -58,6 +58,7 @@ public class BatchExecHashAggregate extends ExecNodeBase private final RowType aggInputRowType; private final boolean isMerge; private final boolean isFinal; +private final boolean supportAdaptiveLocalHashAgg; public BatchExecHashAggregate( ReadableConfig tableConfig, @@ -67,6 +68,7 @@ public class BatchExecHashAggregate extends ExecNodeBase RowType aggInputRowType, boolean isMerge, boolean isFinal, +boolean supportAdaptiveLocalHashAgg, InputProperty inputProperty, RowType outputType, String description) { @@ -83,6 +85,7 @@ public class BatchExecHashAggregate extends ExecNodeBase this.aggInputRowType = aggInputRowType; this.isMerge = isMerge; this.isFinal = isFinal; +this.supportAdaptiveLocalHashAgg = supportAdaptiveLocalHashAgg; } @SuppressWarnings("unchecked") @@ -126,17 +129,17 @@ public class BatchExecHashAggregate extends ExecNodeBase config.get(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_HASH_AGG_MEMORY) .getBytes(); generatedOperator = -new HashAggCodeGenerator( -ctx, -planner.createRelBuilder(), -aggInfos, -inputRowType, -outputRowType, -grouping, -auxGrouping, -isMerge, -isFinal) -.genWithKeys(); +HashAggCodeGenerator.genWithKeys( +ctx, +planner.createRelBuilder(), +aggInfos, +inputRowType, +outputRowType, +grouping, +auxGrouping, +isMerge, +isFinal, +supportAdaptiveLocalHashAgg); } return ExecNodeUtil.createOneInputTransformation( diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/ProjectionCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/ProjectionCodeGenerator.scala index 496af1394fe..e2c3759ddaa 100644 --- a/flink-table/flink-table-
[flink] branch master updated (21e844b6c00 -> dcbb20688c0)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 21e844b6c00 [FLINK-30624][runtime] Introduce a new HeapPriorityQueue for StatusWatermarkValve to avoid affecting the performance of memory state backend. add dcbb20688c0 [FLINK-30727][table-planner] Fix JoinReorderITCaseBase.testBushyTreeJoinReorder failed No new revisions were added by this update. Summary of changes: .../runtime/stream/sql/join/JoinReorderITCase.java | 22 -- .../runtime/utils/JoinReorderITCaseBase.java | 6 ++ 2 files changed, 26 insertions(+), 2 deletions(-)
[flink] branch master updated (2ca7e411f53 -> ec3243e36ee)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 2ca7e411f53 [hotfix] Remove legacy code which was not removed due merge conflicts add ec3243e36ee [FLINK-27920][hive] Supports native avg function for hive dialect No new revisions were added by this update. Summary of changes: .../functions/hive/HiveAverageAggFunction.java | 213 + .../hive/HiveDeclarativeAggregateFunction.java | 16 ++ .../table/functions/hive/HiveSumAggFunction.java | 21 +- .../apache/flink/table/module/hive/HiveModule.java | 15 +- .../connectors/hive/HiveDialectAggITCase.java | 64 +++ .../connectors/hive/HiveDialectQueryPlanTest.java | 14 ++ ...Plan.out => testAvgAggFunctionFallbackPlan.out} | 10 +- ...FunctionPlan.out => testAvgAggFunctionPlan.out} | 10 +- 8 files changed, 334 insertions(+), 29 deletions(-) create mode 100644 flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveAverageAggFunction.java copy flink-connectors/flink-connector-hive/src/test/resources/explain/{testMinAggFunctionFallbackPlan.out => testAvgAggFunctionFallbackPlan.out} (69%) copy flink-connectors/flink-connector-hive/src/test/resources/explain/{testMaxAggFunctionPlan.out => testAvgAggFunctionPlan.out} (53%)
[flink] branch master updated (02b09eac623 -> 916ff76b61a)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 02b09eac623 [FLINK-30707][runtime] Let speculative execution take input data amount into account when detecting slow tasks. add 916ff76b61a [FLINK-30814][table-planner] Fix maxParallelism is not forced to 1 after global partitioner No new revisions were added by this update. Summary of changes: .../planner/plan/nodes/exec/ExecNodeBase.java | 15 +- .../plan/nodes/exec/batch/BatchExecLimit.java | 4 +- .../nodes/exec/batch/BatchExecNestedLoopJoin.java | 3 +- .../plan/nodes/exec/batch/BatchExecRank.java | 4 +- .../nodes/exec/batch/BatchExecScriptTransform.java | 3 +- .../plan/nodes/exec/batch/BatchExecSort.java | 4 +- .../nodes/physical/batch/BatchPhysicalLimit.scala | 8 +- .../physical/batch/BatchPhysicalSortLimit.scala| 12 +- .../runtime/batch/ParallelismSettingTest.java | 255 +++-- 9 files changed, 285 insertions(+), 23 deletions(-)
[flink] branch master updated: [FLINK-29722][hive] Supports native hive max function for hive dialect
This is an automated email from the ASF dual-hosted git repository. godfrey 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 74c7188ae98 [FLINK-29722][hive] Supports native hive max function for hive dialect 74c7188ae98 is described below commit 74c7188ae9898b492c94a472d9d407bf4f8e0876 Author: fengli AuthorDate: Thu Jan 5 21:04:33 2023 +0800 [FLINK-29722][hive] Supports native hive max function for hive dialect This closes #21605 --- .../hive/HiveDeclarativeAggregateFunction.java | 29 + ...MinAggFunction.java => HiveMaxAggFunction.java} | 59 -- .../table/functions/hive/HiveMinAggFunction.java | 19 +--- .../table/functions/hive/HiveSumAggFunction.java | 4 +- .../apache/flink/table/module/hive/HiveModule.java | 6 +- .../connectors/hive/HiveDialectAggITCase.java | 126 ++--- .../connectors/hive/HiveDialectQueryPlanTest.java | 14 +++ .../explain/testMaxAggFunctionFallbackPlan.out | 21 .../resources/explain/testMaxAggFunctionPlan.out | 17 +++ 9 files changed, 227 insertions(+), 68 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveDeclarativeAggregateFunction.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveDeclarativeAggregateFunction.java index 5d316e2e97d..e184207d007 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveDeclarativeAggregateFunction.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveDeclarativeAggregateFunction.java @@ -19,6 +19,7 @@ package org.apache.flink.table.functions.hive; import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.TableException; import org.apache.flink.table.catalog.DataTypeFactory; import org.apache.flink.table.functions.DeclarativeAggregateFunction; import org.apache.flink.table.functions.FunctionKind; @@ -26,9 +27,14 @@ import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.inference.CallContext; import org.apache.flink.table.types.inference.TypeInference; import org.apache.flink.table.types.inference.TypeStrategy; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; +import java.util.List; import java.util.Optional; +import static org.apache.flink.connectors.hive.HiveOptions.TABLE_EXEC_HIVE_NATIVE_AGG_FUNCTION_ENABLED; + /** * API for hive aggregation functions that are expressed in terms of expressions. * @@ -57,6 +63,29 @@ public abstract class HiveDeclarativeAggregateFunction extends DeclarativeAggreg .build(); } +protected void checkArgumentNum(List arguments) { +if (arguments.size() != 1) { +throw new TableException("Exactly one argument is expected."); +} +} + +protected void checkMinMaxArgumentType(LogicalType logicalType, String functionName) { +// Flink doesn't support to compare nested type now, so here can't support it, see +// ScalarOperatorGens#generateComparison for more detail +if (logicalType.is(LogicalTypeRoot.ARRAY) +|| logicalType.is(LogicalTypeRoot.MAP) +|| logicalType.is(LogicalTypeRoot.ROW)) { +throw new TableException( +String.format( +"Native hive %s aggregate function does not support type: %s. " ++ "Please set option '%s' to false to fall back to Hive's own %s function.", +functionName, +logicalType.getTypeRoot(), +TABLE_EXEC_HIVE_NATIVE_AGG_FUNCTION_ENABLED.key(), +functionName)); +} +} + @Override public FunctionKind getKind() { return FunctionKind.AGGREGATE; diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveMinAggFunction.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveMaxAggFunction.java similarity index 60% copy from flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveMinAggFunction.java copy to flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveMaxAggFunction.java index 2eade5a83e6..1e5ddc62e97 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveMinAggFunction.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveMaxAggFunctio
[flink] branch master updated: [FLINK-29719][hive] Supports native count function for hive dialect
This is an automated email from the ASF dual-hosted git repository. godfrey 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 606f297198a [FLINK-29719][hive] Supports native count function for hive dialect 606f297198a is described below commit 606f297198acd74a5c1a39700bd84ad9e26e7b82 Author: fengli AuthorDate: Wed Jan 4 15:31:03 2023 +0800 [FLINK-29719][hive] Supports native count function for hive dialect This closes #21596 --- .../table/functions/hive/HiveCountAggFunction.java | 116 + .../apache/flink/table/module/hive/HiveModule.java | 6 +- .../connectors/hive/HiveDialectAggITCase.java | 72 +++-- .../connectors/hive/HiveDialectQueryPlanTest.java | 24 - .../explain/testCountAggFunctionFallbackPlan.out | 35 +++ .../resources/explain/testCountAggFunctionPlan.out | 27 + 6 files changed, 268 insertions(+), 12 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveCountAggFunction.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveCountAggFunction.java new file mode 100644 index 000..e15a0cbaf3d --- /dev/null +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveCountAggFunction.java @@ -0,0 +1,116 @@ +/* + * 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.functions.hive; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.UnresolvedReferenceExpression; +import org.apache.flink.table.planner.expressions.ExpressionBuilder; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.inference.CallContext; + +import java.util.ArrayList; +import java.util.List; + +import static org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef; +import static org.apache.flink.table.planner.expressions.ExpressionBuilder.ifThenElse; +import static org.apache.flink.table.planner.expressions.ExpressionBuilder.literal; +import static org.apache.flink.table.planner.expressions.ExpressionBuilder.plus; + +/** built-in hive count aggregate function. */ +public class HiveCountAggFunction extends HiveDeclarativeAggregateFunction { + +private final UnresolvedReferenceExpression count = unresolvedRef("count"); +private Integer arguments; +private boolean countLiteral; + +@Override +public int operandCount() { +return arguments; +} + +@Override +public UnresolvedReferenceExpression[] aggBufferAttributes() { +return new UnresolvedReferenceExpression[] {count}; +} + +@Override +public DataType[] getAggBufferTypes() { +return new DataType[] {DataTypes.BIGINT()}; +} + +@Override +public DataType getResultType() { +return DataTypes.BIGINT(); +} + +@Override +public Expression[] initialValuesExpressions() { +return new Expression[] {/* count = */ literal(0L, getResultType().notNull())}; +} + +@Override +public Expression[] accumulateExpressions() { +// count(*) and count(literal) mean that count all elements +if (arguments == 0 || countLiteral) { +return new Expression[] {/* count = */ plus(count, literal(1L))}; +} + +// other case need to determine the value of the element +List operandExpressions = new ArrayList<>(); +for (int i = 0; i < arguments; i++) { +operandExpressions.add(operand(i)); +} +Expression operandExpression = +operandExpressions.stream() +.map(ExpressionBuilder::isNull) +.reduce(ExpressionBuilder::or) +.get(); +return new Expression[] { +/* count = */ ifThenElse(operandExpression, count, plus(count, literal(1L))) +}; +}
[flink] branch master updated: [FLINK-29721][hive] Supports native hive min function for hive dialect
This is an automated email from the ASF dual-hosted git repository. godfrey 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 8a1f66ca827 [FLINK-29721][hive] Supports native hive min function for hive dialect 8a1f66ca827 is described below commit 8a1f66ca827163b32387e0043f4362921f6c11a9 Author: Tartarus0zm AuthorDate: Fri Jan 6 16:09:38 2023 +0800 [FLINK-29721][hive] Supports native hive min function for hive dialect This closes #21608 --- .../table/functions/hive/HiveMinAggFunction.java | 126 .../apache/flink/table/module/hive/HiveModule.java | 25 +++- .../connectors/hive/HiveDialectAggITCase.java | 132 + .../connectors/hive/HiveDialectQueryPlanTest.java | 121 +++ .../explain/testMinAggFunctionFallbackPlan.out | 21 .../resources/explain/testMinAggFunctionPlan.out | 17 +++ 6 files changed, 419 insertions(+), 23 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveMinAggFunction.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveMinAggFunction.java new file mode 100644 index 000..2eade5a83e6 --- /dev/null +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveMinAggFunction.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.functions.hive; + +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.UnresolvedReferenceExpression; +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.logical.LogicalTypeRoot; + +import static org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef; +import static org.apache.flink.table.planner.expressions.ExpressionBuilder.ifThenElse; +import static org.apache.flink.table.planner.expressions.ExpressionBuilder.isNull; +import static org.apache.flink.table.planner.expressions.ExpressionBuilder.lessThan; +import static org.apache.flink.table.planner.expressions.ExpressionBuilder.nullOf; + +/** built-in hive min aggregate function. */ +public class HiveMinAggFunction extends HiveDeclarativeAggregateFunction { + +private final UnresolvedReferenceExpression min = unresolvedRef("min"); +private DataType resultType; + +@Override +public int operandCount() { +return 1; +} + +@Override +public UnresolvedReferenceExpression[] aggBufferAttributes() { +return new UnresolvedReferenceExpression[] {min}; +} + +@Override +public DataType[] getAggBufferTypes() { +return new DataType[] {getResultType()}; +} + +@Override +public DataType getResultType() { +return resultType; +} + +@Override +public Expression[] initialValuesExpressions() { +return new Expression[] { +/* min */ +nullOf(getResultType()) +}; +} + +@Override +public Expression[] accumulateExpressions() { +return new Expression[] { +/* min = */ ifThenElse( +isNull(operand(0)), +min, +ifThenElse( +isNull(min), +operand(0), +ifThenElse(lessThan(operand(0), min), operand(0), min))) +}; +} + +@Override +public Expression[] retractExpressions() { +throw new TableException("Min aggregate function does not support retraction."); +} + +@Override +public Expression[] mergeExpressions() { +return new Expression[] { +/* min = */ ifThenElse( +isNull(mergeOperand(min)), +min, +ifThenElse(
[flink] branch master updated (128b0c4442a -> 14ea1a58e9b)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 128b0c4442a [FLINK-30675][sql-client] Decompose query result print from Executor add 14ea1a58e9b [FLINK-30376][table-planner] Introduce bushy join reorder rule based on greedy algorithm No new revisions were added by this update. Summary of changes: .../generated/optimizer_config_configuration.html |6 + .../table/api/config/OptimizerConfigOptions.java | 12 + .../rules/logical/FlinkBushyJoinReorderRule.java | 642 ++ .../plan/rules/logical/FlinkJoinReorderRule.java | 102 + .../planner/plan/rules/FlinkBatchRuleSets.scala|2 +- .../planner/plan/rules/FlinkStreamRuleSets.scala |4 +- .../logical/RewriteMultiJoinConditionRule.scala| 17 +- .../planner/utils/JavaScalaConversionUtil.scala|3 + .../DynamicPartitionPruningProgramTest.java| 108 - .../runtime/batch/sql/join/JoinReorderITCase.java | 64 + .../runtime/stream/sql/join/JoinReorderITCase.java | 74 + .../runtime/utils/JoinReorderITCaseBase.java | 366 .../plan/batch/sql/join/JoinReorderTest.xml| 2134 +++ .../program/DynamicPartitionPruningProgramTest.xml | 133 +- .../logical/RewriteMultiJoinConditionRuleTest.xml | 48 + .../plan/stream/sql/join/JoinReorderTest.xml | 2200 .../plan/batch/sql/join/JoinReorderTest.scala |7 +- .../planner/plan/common/JoinReorderTestBase.scala | 158 +- .../RewriteMultiJoinConditionRuleTest.scala| 22 + .../plan/stream/sql/join/JoinReorderTest.scala |8 +- .../runtime/batch/sql/join/JoinITCase.scala| 211 -- 21 files changed, 5047 insertions(+), 1274 deletions(-) create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkBushyJoinReorderRule.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinReorderRule.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/join/JoinReorderITCase.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/join/JoinReorderITCase.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/utils/JoinReorderITCaseBase.java
[flink] branch release-1.16 updated (85248af102f -> 54518e9e27c)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git from 85248af102f [FLINK-27327][docs] Add description about changing max parallelism explicitly leads to state incompatibility add 54518e9e27c [FLINK-30586][table-runtime] Fix calcCodeGen failed if calc with 'like' condition contains double quotation mark No new revisions were added by this update. Summary of changes: .../table/planner/codegen/GenerateUtils.scala | 7 +++-- .../planner/runtime/batch/sql/CalcITCase.scala | 29 ++-- .../planner/runtime/stream/sql/CalcITCase.scala| 32 +- 3 files changed, 61 insertions(+), 7 deletions(-)
[flink] branch master updated: [FLINK-30586][table-runtime] Fix calcCodeGen failed if calc with 'like' condition contains double quotation mark
This is an automated email from the ASF dual-hosted git repository. godfrey 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 68b37fb8673 [FLINK-30586][table-runtime] Fix calcCodeGen failed if calc with 'like' condition contains double quotation mark 68b37fb8673 is described below commit 68b37fb867374df5a201f0b170e35c21266e5d7b Author: zhengyunhong.zyh <337361...@qq.com> AuthorDate: Fri Jan 6 16:25:32 2023 +0800 [FLINK-30586][table-runtime] Fix calcCodeGen failed if calc with 'like' condition contains double quotation mark This closes #21609 --- .../table/planner/codegen/GenerateUtils.scala | 7 +++-- .../planner/runtime/batch/sql/CalcITCase.scala | 29 ++-- .../planner/runtime/stream/sql/CalcITCase.scala| 32 +- 3 files changed, 61 insertions(+), 7 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala index 9708ca70e66..92a45ab14b6 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala @@ -304,9 +304,10 @@ object GenerateUtils { // as they're not cheap to construct. For the other types, the return term is directly // the literal value case CHAR | VARCHAR => -val str = literalValue.asInstanceOf[BinaryStringData] -val field = ctx.addReusableEscapedStringConstant(EncodingUtils.escapeJava(str.toString)) -generateNonNullLiteral(literalType, field, str) +val escapedValue = + EncodingUtils.escapeJava(literalValue.asInstanceOf[BinaryStringData].toString) +val field = ctx.addReusableEscapedStringConstant(escapedValue) +generateNonNullLiteral(literalType, field, StringData.fromString(escapedValue)) case BINARY | VARBINARY => val bytesVal = literalValue.asInstanceOf[Array[Byte]] diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala index b95df8c1a89..ef4ebd507f2 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala @@ -50,11 +50,9 @@ import org.junit.rules.ExpectedException import java.nio.charset.StandardCharsets import java.sql.{Date, Time, Timestamp} -import java.time.{Instant, LocalDate, LocalDateTime, LocalTime, ZoneId} +import java.time._ import java.util -import scala.collection.Seq - class CalcITCase extends BatchTestBase { var _expectedEx: ExpectedException = ExpectedException.none @@ -2115,4 +2113,29 @@ class CalcITCase extends BatchTestBase { tEnv.useDatabase("db1") checkResult("SELECT CURRENT_DATABASE()", Seq(row(tEnv.getCurrentDatabase))) } + + @Test + def testLikeWithConditionContainsDoubleQuotationMark(): Unit = { +val rows = Seq(row(42, "abc"), row(2, "cbc\"ddd")) +val dataId = TestValuesTableFactory.registerData(rows) + +val ddl = + s""" + |CREATE TABLE MyTable ( + | a int, + | b string + |) WITH ( + | 'connector' = 'values', + | 'data-id' = '$dataId', + | 'bounded' = 'true' + |) + |""".stripMargin +tEnv.executeSql(ddl) + +checkResult( + """ +| SELECT * FROM MyTable WHERE b LIKE '%"%' +|""".stripMargin, + Seq(row(2, "cbc\"ddd"))) + } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala index 1381851eb40..886a862d867 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala @@ -46,7 +46,6 @@ import java.time.Instant import java.util import scala.collection.JavaConversions._ -import scala.collection.Seq class CalcITCase extends StreamingTestBase { @@ -714,4 +713,35 @@ class CalcITCase extends StreamingTestBase { val result2 = tEnv.sqlQuery("SELECT CURRENT_DATABASE()").execute().col
[flink] branch master updated: [FLINK-30491][hive] Hive table partition supports lazy deserialization during runtime
This is an automated email from the ASF dual-hosted git repository. godfrey 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 d4340d16708 [FLINK-30491][hive] Hive table partition supports lazy deserialization during runtime d4340d16708 is described below commit d4340d16708010394d7c57063b5dece8362d41d0 Author: fengli AuthorDate: Sat Dec 24 16:46:03 2022 +0800 [FLINK-30491][hive] Hive table partition supports lazy deserialization during runtime This closes #21556 --- .../apache/flink/connectors/hive/HiveSource.java | 8 +-- .../flink/connectors/hive/HiveSourceBuilder.java | 13 +++- .../hive/HiveSourceDynamicFileEnumerator.java | 10 ++-- .../connectors/hive/HiveSourceFileEnumerator.java | 13 ++-- .../hive/HiveTablePartitionSerializer.java | 68 + .../connectors/hive/util/HivePartitionUtils.java | 30 ++ .../hive/util/HivePartitionUtilsTest.java | 70 ++ 7 files changed, 197 insertions(+), 15 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSource.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSource.java index d340ed616ea..5b768b0320a 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSource.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSource.java @@ -64,7 +64,7 @@ public class HiveSource extends AbstractFileSource { private final String hiveVersion; private final List dynamicFilterPartitionKeys; -private final List partitions; +private final List partitionBytes; private final ContinuousPartitionFetcher fetcher; private final HiveTableSource.HiveContinuousPartitionFetcherContext fetcherContext; private final ObjectPath tablePath; @@ -80,7 +80,7 @@ public class HiveSource extends AbstractFileSource { List partitionKeys, String hiveVersion, @Nullable List dynamicFilterPartitionKeys, -List partitions, +List partitionBytes, @Nullable ContinuousPartitionFetcher fetcher, @Nullable HiveTableSource.HiveContinuousPartitionFetcherContext fetcherContext) { super( @@ -94,7 +94,7 @@ public class HiveSource extends AbstractFileSource { this.partitionKeys = partitionKeys; this.hiveVersion = hiveVersion; this.dynamicFilterPartitionKeys = dynamicFilterPartitionKeys; -this.partitions = partitions; +this.partitionBytes = partitionBytes; this.fetcher = fetcher; this.fetcherContext = fetcherContext; } @@ -181,7 +181,7 @@ public class HiveSource extends AbstractFileSource { new HiveSourceDynamicFileEnumerator.Provider( tablePath.getFullName(), dynamicFilterPartitionKeys, -partitions, +partitionBytes, hiveVersion, jobConfWrapper), getAssignerFactory()); diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceBuilder.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceBuilder.java index bb6ee5a6c1e..86de17520a9 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceBuilder.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceBuilder.java @@ -234,11 +234,18 @@ public class HiveSourceBuilder { continuousSourceSettings == null || partitionKeys.isEmpty() ? DEFAULT_SPLIT_ASSIGNER : SimpleSplitAssigner::new; +List hiveTablePartitionBytes = Collections.emptyList(); +if (partitions != null) { +// Serializing the HiveTablePartition list manually at compile time to avoid +// deserializing it in TaskManager during runtime. The HiveTablePartition list is no +// need for TM. +hiveTablePartitionBytes = HivePartitionUtils.serializeHiveTablePartition(partitions); +} + return new HiveSource<>( new Path[1], new HiveSourceFileEnumerator.Provider( -partitions != null ? partitions : Collections.emptyList(), -new JobConfWrapper(jobConf)), +hiveTablePartitionBytes, new JobConfWrapper(jobConf)), splitAssigner, bulkFormat, continuousSourceSettings, @@ -247,7 +254,7 @@ public class HiveSourceB
[flink] branch master updated: [FLINK-30365][table-planner] New dynamic partition pruning strategy to support more dpp patterns
This is an automated email from the ASF dual-hosted git repository. godfrey 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 d9f9b55f82d [FLINK-30365][table-planner] New dynamic partition pruning strategy to support more dpp patterns d9f9b55f82d is described below commit d9f9b55f82dfbc1676572cc36b718a99001497f8 Author: zhengyunhong.zyh <337361...@qq.com> AuthorDate: Mon Dec 12 12:06:38 2022 +0800 [FLINK-30365][table-planner] New dynamic partition pruning strategy to support more dpp patterns This closes #21489 --- .../FlinkDynamicPartitionPruningProgram.java | 148 + .../batch/DynamicPartitionPruningRule.java | 739 - .../utils/DynamicPartitionPruningUtils.java| 699 +++ .../planner/plan/metadata/FlinkRelMdRowCount.scala | 18 +- .../plan/optimize/program/FlinkBatchProgram.scala | 4 + .../planner/plan/rules/FlinkBatchRuleSets.scala| 21 +- .../DynamicPartitionPruningProgramTest.java} | 196 +- .../DynamicPartitionPruningProgramTest.xml}| 375 +-- 8 files changed, 1102 insertions(+), 1098 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/FlinkDynamicPartitionPruningProgram.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/FlinkDynamicPartitionPruningProgram.java new file mode 100644 index 000..22aebd7fcab --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/FlinkDynamicPartitionPruningProgram.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.optimize.program; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.table.api.config.OptimizerConfigOptions; +import org.apache.flink.table.planner.plan.nodes.physical.batch.BatchPhysicalDynamicFilteringTableSourceScan; +import org.apache.flink.table.planner.plan.nodes.physical.batch.BatchPhysicalTableSourceScan; +import org.apache.flink.table.planner.plan.utils.DefaultRelShuttle; +import org.apache.flink.table.planner.utils.DynamicPartitionPruningUtils; +import org.apache.flink.table.planner.utils.ShortcutUtils; + +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Join; +import org.apache.calcite.rel.core.JoinInfo; +import org.apache.calcite.rel.core.JoinRelType; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * Planner program that tries to do partition prune in the execution phase, which can translate a + * {@link BatchPhysicalTableSourceScan} to a {@link BatchPhysicalDynamicFilteringTableSourceScan} + * whose source is a partition source. The {@link + * OptimizerConfigOptions#TABLE_OPTIMIZER_DYNAMIC_FILTERING_ENABLED} need to be true. + * + * Suppose we have the original physical plan: + * + * {@code + * LogicalProject(...) + * HashJoin(joinType=[InnerJoin], where=[=(fact_partition_key, dim_key)], select=[xxx]) + * * :- TableSourceScan(table=[[fact]], fields=[xxx, fact_partition_key],) # Is a partition table. + * * +- Exchange(distribution=[broadcast]) + * *+- Calc(select=[xxx], where=[<(xxx, xxx)]) # Need have an arbitrary filter condition. + * * +- TableSourceScan(table=[[dim, filter=[]]], fields=[xxx, dim_key]) + * } + * + * This physical plan will be rewritten to: + * + * {@code + * HashJoin(joinType=[InnerJoin], where=[=(fact_partition_key, dim_key)], select=[xxx]) + * :- DynamicFilteringTableSourceScan(table=[[fact]], fields=[xxx, fact_partition_key]) # Is a partition table. + * : +- DynamicFilteringDataCollector(fields=[dim_key]) + * : +- Calc(select=[xxx], where=[<(xxx, xxx)]) + * :+- TableSourceScan(table=[[dim, filter=[]]], fields=[xxx, dim_key]) + * +- Exchange(distribution=[broadcast]) + *+- Calc(select=[xxx], where=[<(xxx, xxx)]) # Need have an arbitrary filter condition. + * +- TableSourceScan(table=[[dim
[flink] branch release-1.16 updated (e7f7e6b8c17 -> dbb6654c9d2)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git from e7f7e6b8c17 [FLINK-30539][tests] Removes timeout from YARNSessionCapacitySchedulerITCase add dbb6654c9d2 [FLINK-29849][table-planner] Fix event time temporal join on an upsert source may produce incorrect execution plan No new revisions were added by this update. Summary of changes: .../logical/EventTimeTemporalJoinRewriteRule.java | 474 + .../plan/rules/logical/FlinkFilterJoinRule.java| 2 + .../planner/plan/utils/TemporalTableJoinUtil.java | 69 +++ .../logical/FlinkLogicalTableSourceScan.scala | 30 +- .../stream/StreamPhysicalTemporalJoin.scala| 4 +- .../plan/optimize/program/FlinkStreamProgram.scala | 19 +- .../planner/plan/rules/FlinkStreamRuleSets.scala | 2 +- .../TemporalJoinRewriteWithUniqueKeyRule.scala | 7 +- .../stream/StreamPhysicalTableSourceScanRule.scala | 4 +- .../planner/plan/utils/TemporalJoinUtil.scala | 7 +- ...AssignerChangelogNormalizeTransposeRuleTest.xml | 8 +- .../planner/plan/stream/sql/TableScanTest.xml | 8 +- .../plan/stream/sql/join/TemporalJoinTest.xml | 38 ++ ...signerChangelogNormalizeTransposeRuleTest.scala | 2 + .../plan/stream/sql/join/TemporalJoinTest.scala| 64 ++- .../runtime/stream/sql/TemporalJoinITCase.scala| 5 +- .../temporal/TemporalRowTimeJoinOperatorTest.java | 77 17 files changed, 787 insertions(+), 33 deletions(-) create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/EventTimeTemporalJoinRewriteRule.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/TemporalTableJoinUtil.java
[flink] branch release-1.16 updated (31f1e235f87 -> c1435524399)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git from 31f1e235f87 [FLINK-27691][table-planner] Remove unnecessary state ttl to avoid unstable results of RankHarnessTest add 17b42516ceb [FLINK-28988][table-planner] Don't push filters down into the right table for temporal join add c1435524399 [hotfix][table-planner] Fix tests for FLINK-28988 merged No new revisions were added by this update. Summary of changes: .../plan/rules/logical/FlinkFilterJoinRule.java| 36 +++ .../physical/stream/ChangelogModeInferenceTest.xml | 108 + ...AssignerChangelogNormalizeTransposeRuleTest.xml | 15 ++- .../plan/stream/sql/join/TemporalJoinTest.xml | 64 ++-- .../stream/ChangelogModeInferenceTest.scala| 33 +++ .../runtime/stream/sql/TemporalJoinITCase.scala| 26 - 6 files changed, 235 insertions(+), 47 deletions(-)
[flink] branch master updated (94fba321d4e -> be6b1c94ef3)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 94fba321d4e [FLINK-22318][table] Support RENAME column name for ALTER TABLE statement, optimize the code implementation new 62a9d837e24 [FLINK-28850][table-planner] pre-step: copy Snapshot node from calcite to support alias of LOOKUP hint new f32ba645246 [FLINK-28850][table-planner] cherry-pick CALCITE-5251 to support hint for Snapshot node new be6b1c94ef3 [FLINK-28850][table-planner] Support table alias in LOOKUP hint 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: .../content.zh/docs/dev/table/sql/queries/hints.md | 2 +- docs/content/docs/dev/table/sql/queries/hints.md | 2 +- .../java/org/apache/calcite/rel/core/Snapshot.java | 134 +++ .../apache/calcite/rel/hint/HintPredicates.java| 7 + .../calcite/rel/hint/NodeTypeHintPredicate.java| 6 +- .../calcite/rel/logical/LogicalSnapshot.java | 34 ++- .../flink/table/planner/hint/FlinkHints.java | 1 - .../nodes/exec/stream/LookupJoinJsonPlanTest.java | 10 +- .../plan/stream/sql/join/LookupJoinTest.xml| 260 + .../plan/stream/sql/join/LookupJoinTest.scala | 66 -- .../runtime/stream/sql/AsyncLookupJoinITCase.scala | 6 +- .../runtime/stream/sql/LookupJoinITCase.scala | 8 +- 12 files changed, 452 insertions(+), 84 deletions(-) create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Snapshot.java
[flink] 03/03: [FLINK-28850][table-planner] Support table alias in LOOKUP hint
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit be6b1c94ef3f552c753746863cb0a4e7dd86d2fc Author: lincoln lee AuthorDate: Fri Sep 9 17:48:04 2022 +0800 [FLINK-28850][table-planner] Support table alias in LOOKUP hint This closes #20800 --- .../content.zh/docs/dev/table/sql/queries/hints.md | 2 +- docs/content/docs/dev/table/sql/queries/hints.md | 2 +- .../flink/table/planner/hint/FlinkHints.java | 1 - .../nodes/exec/stream/LookupJoinJsonPlanTest.java | 10 +- .../plan/stream/sql/join/LookupJoinTest.xml| 260 + .../plan/stream/sql/join/LookupJoinTest.scala | 66 -- .../runtime/stream/sql/AsyncLookupJoinITCase.scala | 6 +- .../runtime/stream/sql/LookupJoinITCase.scala | 8 +- 8 files changed, 275 insertions(+), 80 deletions(-) diff --git a/docs/content.zh/docs/dev/table/sql/queries/hints.md b/docs/content.zh/docs/dev/table/sql/queries/hints.md index 9e6ce8b3cce..7cfec52207b 100644 --- a/docs/content.zh/docs/dev/table/sql/queries/hints.md +++ b/docs/content.zh/docs/dev/table/sql/queries/hints.md @@ -331,7 +331,7 @@ LOOKUP 联接提示允许用户建议 Flink 优化器: {{< hint info >}} 注意:其中 -- 'table' 是必选项,需要填写目标联接表的表名(和 FROM 子句引用的表名保持一致),注意当前不支持填写表的别名(这将在后续版本中支持)。 +- 'table' 是必选项,需要填写目标联接表的表名(和 FROM 子句引用的表名保持一致),注意如果表定义了别名,则提示选项必须使用别名。 - 异步查找参数可按需设置一个或多个,未设置的参数按默认值生效。 - 重试查找参数没有默认值,在需要开启时所有参数都必须设置为有效值。 {{< /hint >}} diff --git a/docs/content/docs/dev/table/sql/queries/hints.md b/docs/content/docs/dev/table/sql/queries/hints.md index 5cc8723112e..33eebcfb75a 100644 --- a/docs/content/docs/dev/table/sql/queries/hints.md +++ b/docs/content/docs/dev/table/sql/queries/hints.md @@ -338,7 +338,7 @@ The LOOKUP hint allows users to suggest the Flink optimizer to: {{< hint info >}} Note: -- 'table' option is required, only table name is supported(keep consistent with which in the FROM clause), alias name is not supported currently(will be supported in later versions). +- 'table' option is required, only table name is supported(keep consistent with which in the FROM clause), note that only alias name can be used if table has an alias name. - async options are all optional, will use default value if not configured. - there is no default value for retry options, all retry options should be set to valid values when need to enable retry. {{< /hint >}} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/FlinkHints.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/FlinkHints.java index 7463ac22893..4e8ce7a7fcb 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/FlinkHints.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/FlinkHints.java @@ -114,7 +114,6 @@ public abstract class FlinkHints { public static boolean canTransposeToTableScan(RelNode node) { return node instanceof LogicalProject // computed column on table || node instanceof LogicalFilter -// TODO support lookup join hint with alias name in FLINK-28850 || node instanceof LogicalSnapshot; } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest.java index 05b9200280a..a68eee6357a 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest.java @@ -201,7 +201,7 @@ public class LookupJoinJsonPlanTest extends TableTestBase { // LookupTable has sync func only, just verify the hint has take effect util.verifyJsonPlan( "INSERT INTO MySink1 SELECT " -+ "/*+ LOOKUP('table'='LookupTable', 'async'='true', 'output-mode'='allow_unordered') */ * " ++ "/*+ LOOKUP('table'='D', 'async'='true', 'output-mode'='allow_unordered') */ * " + "FROM MyTable AS T JOIN LookupTable " + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id"); } @@ -211,7 +211,7 @@ public class LookupJoinJsonPlanTest extends TableTestBase { // LookupTable has sync func only, just verify the hint has take effect util.verifyJsonPlan( "INSERT INTO MySink1 SELECT " -+ "/*+ LOOKUP('table'='LookupTable', 'async'='true', 'timeout'='600s', 'capacity'='1000') */
[flink] 01/03: [FLINK-28850][table-planner] pre-step: copy Snapshot node from calcite to support alias of LOOKUP hint
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 62a9d837e24150461790e7689659c50f15197ebc Author: lincoln lee AuthorDate: Wed Dec 28 11:10:20 2022 +0800 [FLINK-28850][table-planner] pre-step: copy Snapshot node from calcite to support alias of LOOKUP hint This closes #20800 --- .../java/org/apache/calcite/rel/core/Snapshot.java | 104 + 1 file changed, 104 insertions(+) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Snapshot.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Snapshot.java new file mode 100644 index 000..b47aa300d37 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Snapshot.java @@ -0,0 +1,104 @@ +/* + * 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.calcite.rel.core; + +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.RelWriter; +import org.apache.calcite.rel.SingleRel; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexShuttle; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.calcite.util.Litmus; +import org.checkerframework.checker.nullness.qual.Nullable; + +import java.util.List; +import java.util.Objects; + +/** + * Relational expression that returns the contents of a relation expression as it was at a given + * time in the past. + * + * For example, if {@code Products} is a temporal table, and {@link TableScan}(Products) is a + * relational operator that returns all versions of the contents of the table, then {@link + * Snapshot}(TableScan(Products)) is a relational operator that only returns the contents whose + * versions that overlap with the given specific period (i.e. those that started before given period + * and ended after it). + */ +public abstract class Snapshot extends SingleRel { +// ~ Instance fields + +private final RexNode period; + +// ~ Constructors --- + +/** + * Creates a Snapshot. + * + * @param cluster Cluster that this relational expression belongs to + * @param traitSet The traits of this relational expression + * @param input Input relational expression + * @param period Timestamp expression which as the table was at the given time in the past + */ +@SuppressWarnings("method.invocation.invalid") +protected Snapshot(RelOptCluster cluster, RelTraitSet traitSet, RelNode input, RexNode period) { +super(cluster, traitSet, input); +this.period = Objects.requireNonNull(period, "period"); +assert isValid(Litmus.THROW, null); +} + +// ~ Methods + +@Override +public final RelNode copy(RelTraitSet traitSet, List inputs) { +return copy(traitSet, sole(inputs), getPeriod()); +} + +public abstract Snapshot copy(RelTraitSet traitSet, RelNode input, RexNode period); + +@Override +public RelNode accept(RexShuttle shuttle) { +RexNode condition = shuttle.apply(this.period); +if (this.period == condition) { +return this; +} +return copy(traitSet, getInput(), condition); +} + +@Override +public RelWriter explainTerms(RelWriter pw) { +return super.explainTerms(pw).item("period", period); +} + +public RexNode getPeriod() { +return period; +} + +@Override +public boolean isValid(Litmus litmus, @Nullable Context context) { +RelDataType dataType = period.getType(); +if (dataType.getSqlTypeName() != SqlTypeName.TIMESTAMP) { +return litmus.fail( +"The system time period specification expects Timestamp type but is '&
[flink] 02/03: [FLINK-28850][table-planner] cherry-pick CALCITE-5251 to support hint for Snapshot node
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit f32ba645246a8180a02182650fb51392facfcc09 Author: lincoln lee AuthorDate: Fri Sep 9 16:44:03 2022 +0800 [FLINK-28850][table-planner] cherry-pick CALCITE-5251 to support hint for Snapshot node This closes #20800 --- .../java/org/apache/calcite/rel/core/Snapshot.java | 42 ++ .../apache/calcite/rel/hint/HintPredicates.java| 7 .../calcite/rel/hint/NodeTypeHintPredicate.java| 6 +++- .../calcite/rel/logical/LogicalSnapshot.java | 34 -- 4 files changed, 79 insertions(+), 10 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Snapshot.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Snapshot.java index b47aa300d37..531f69345dc 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Snapshot.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Snapshot.java @@ -16,17 +16,19 @@ */ package org.apache.calcite.rel.core; +import com.google.common.collect.ImmutableList; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelTraitSet; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.RelWriter; import org.apache.calcite.rel.SingleRel; +import org.apache.calcite.rel.hint.Hintable; +import org.apache.calcite.rel.hint.RelHint; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rex.RexNode; import org.apache.calcite.rex.RexShuttle; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.util.Litmus; -import org.checkerframework.checker.nullness.qual.Nullable; import java.util.List; import java.util.Objects; @@ -40,12 +42,17 @@ import java.util.Objects; * Snapshot}(TableScan(Products)) is a relational operator that only returns the contents whose * versions that overlap with the given specific period (i.e. those that started before given period * and ended after it). + * + * Temporarily copy from calcite to cherry-pick [CALCITE-5251] and will be removed when upgrade + * to caclite-1.32.0. */ -public abstract class Snapshot extends SingleRel { +public abstract class Snapshot extends SingleRel implements Hintable { // ~ Instance fields private final RexNode period; +protected final ImmutableList hints; + // ~ Constructors --- /** @@ -53,16 +60,35 @@ public abstract class Snapshot extends SingleRel { * * @param cluster Cluster that this relational expression belongs to * @param traitSet The traits of this relational expression + * @param hints Hints for this node * @param input Input relational expression * @param period Timestamp expression which as the table was at the given time in the past */ -@SuppressWarnings("method.invocation.invalid") -protected Snapshot(RelOptCluster cluster, RelTraitSet traitSet, RelNode input, RexNode period) { +protected Snapshot( +RelOptCluster cluster, +RelTraitSet traitSet, +List hints, +RelNode input, +RexNode period) { super(cluster, traitSet, input); this.period = Objects.requireNonNull(period, "period"); +this.hints = ImmutableList.copyOf(hints); assert isValid(Litmus.THROW, null); } +/** + * Creates a Snapshot. + * + * @param cluster Cluster that this relational expression belongs to + * @param traitSet The traits of this relational expression + * @param input Input relational expression + * @param period Timestamp expression which as the table was at the given time in the past + */ +@SuppressWarnings("method.invocation.invalid") +protected Snapshot(RelOptCluster cluster, RelTraitSet traitSet, RelNode input, RexNode period) { +this(cluster, traitSet, ImmutableList.of(), input, period); +} + // ~ Methods @Override @@ -72,7 +98,6 @@ public abstract class Snapshot extends SingleRel { public abstract Snapshot copy(RelTraitSet traitSet, RelNode input, RexNode period); -@Override public RelNode accept(RexShuttle shuttle) { RexNode condition = shuttle.apply(this.period); if (this.period == condition) { @@ -91,7 +116,7 @@ public abstract class Snapshot extends SingleRel { } @Override -public boolean isValid(Litmus litmus, @Nullable Context context) { +public boolean isValid(Litmus litmus, RelNode.Context context) { RelDataType dataType = period.getType();
[flink] branch master updated: [FLINK-30368][table-planner] Fix calcite method RelMdUtil$numDistinctVals() wrongly return zero if the method input domainSize much larger than numSelected
This is an automated email from the ASF dual-hosted git repository. godfrey 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 dc862dae28a [FLINK-30368][table-planner] Fix calcite method RelMdUtil$numDistinctVals() wrongly return zero if the method input domainSize much larger than numSelected dc862dae28a is described below commit dc862dae28a172f674a9b8a2198c603275304550 Author: zhengyunhong.zyh <337361...@qq.com> AuthorDate: Mon Dec 12 14:44:59 2022 +0800 [FLINK-30368][table-planner] Fix calcite method RelMdUtil$numDistinctVals() wrongly return zero if the method input domainSize much larger than numSelected This closes #21490 --- .../plan/metadata/FlinkRelMdDistinctRowCount.scala | 2 +- .../table/planner/plan/utils/FlinkRelMdUtil.scala | 81 ++ .../planner/plan/utils/FlinkRelMdUtilTest.scala| 1 + 3 files changed, 83 insertions(+), 1 deletion(-) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCount.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCount.scala index f74b7b0473b..86158e0c85a 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCount.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCount.scala @@ -539,7 +539,7 @@ class FlinkRelMdDistinctRowCount private extends MetadataHandler[BuiltInMetadata } mq.getDistinctRowCount(rel.getLeft, groupKey, newPred) case _ => -RelMdUtil.getJoinDistinctRowCount(mq, rel, rel.getJoinType, groupKey, predicate, false) +FlinkRelMdUtil.getJoinDistinctRowCount(mq, rel, rel.getJoinType, groupKey, predicate, false) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelMdUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelMdUtil.scala index bab6f791433..a2038d0897b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelMdUtil.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelMdUtil.scala @@ -37,6 +37,7 @@ import org.apache.calcite.rex._ import org.apache.calcite.sql.`type`.SqlTypeName.{TIME, TIMESTAMP} import org.apache.calcite.sql.SqlKind import org.apache.calcite.util.{ImmutableBitSet, NumberUtil} +import org.apache.calcite.util.NumberUtil.multiply import java.math.BigDecimal import java.util @@ -247,6 +248,86 @@ object FlinkRelMdUtil { RexUtil.composeConjunction(rexBuilder, pushable, true) } + /** + * This method is copied from calcite RelMdUtil and line 324 ~ 328 are changed. This method should + * be removed once CALCITE-4351 is fixed. See CALCITE-4351 and FLINK-19780. + * + * Computes the number of distinct rows for a set of keys returned from a join. Also known as NDV + * (number of distinct values). + * + * @param joinRel + * RelNode representing the join + * @param joinType + * type of join + * @param groupKey + * keys that the distinct row count will be computed for + * @param predicate + * join predicate + * @param useMaxNdv + * If true use formula max(left NDV, right NDV), otherwise use left NDV * + * right NDV. + * @return + * number of distinct rows + */ + def getJoinDistinctRowCount( + mq: RelMetadataQuery, + joinRel: RelNode, + joinType: JoinRelType, + groupKey: ImmutableBitSet, + predicate: RexNode, + useMaxNdv: Boolean): JDouble = { +if ((predicate == null || predicate.isAlwaysTrue) && groupKey.isEmpty) { + return 1d +} +val join = joinRel.asInstanceOf[Join] +if (join.isSemiJoin) { + return RelMdUtil.getSemiJoinDistinctRowCount(join, mq, groupKey, predicate) +} +val leftMask = ImmutableBitSet.builder +val rightMask = ImmutableBitSet.builder +val left = joinRel.getInputs.get(0) +val right = joinRel.getInputs.get(1) +RelMdUtil.setLeftRightBitmaps(groupKey, leftMask, rightMask, left.getRowType.getFieldCount) +// determine which filters apply to the left vs right +val (leftPred, rightPred) = if (predicate != null) { + val leftFilters = new util.ArrayList[RexNode] + val rightFilters = new util.ArrayList[RexNode] + val joinFilters = new util.ArrayList[RexNode] + val predList = RelOptUtil.conjunctions(predicate) + RelOptUtil.classifyFilters( +joinRel, +predList, +joinType.canPushIntoFromAbove, +joinType.canPushLeftFromAbove, +joinType.canPushRightFromAbove, +
[flink] branch release-1.16 updated: [FLINK-30270][table-planner] Fix FlinkJoinToMultiJoinRule incorrectly combines Left/Right outer join to MultiJoin error
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.16 by this push: new ca42695dba5 [FLINK-30270][table-planner] Fix FlinkJoinToMultiJoinRule incorrectly combines Left/Right outer join to MultiJoin error ca42695dba5 is described below commit ca42695dba5ab72c6b9b895bb6553321c30d5074 Author: zhengyunhong.zyh <337361...@qq.com> AuthorDate: Tue Dec 27 14:21:24 2022 +0800 [FLINK-30270][table-planner] Fix FlinkJoinToMultiJoinRule incorrectly combines Left/Right outer join to MultiJoin error This closes #21487 --- .../rules/logical/FlinkJoinToMultiJoinRule.java| 215 -- .../plan/batch/sql/join/JoinReorderTest.xml| 53 ++-- .../rules/logical/FlinkJoinToMultiJoinRuleTest.xml | 316 + .../logical/FlinkJoinToMultiJoinRuleTest.scala | 91 -- 4 files changed, 475 insertions(+), 200 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRule.java index 838062dc15c..0766c1d4699 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRule.java @@ -18,11 +18,14 @@ package org.apache.flink.table.planner.plan.rules.logical; +import org.apache.flink.table.api.TableException; + import org.apache.calcite.plan.RelOptRuleCall; import org.apache.calcite.plan.RelOptUtil; import org.apache.calcite.plan.RelRule; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.core.Join; +import org.apache.calcite.rel.core.JoinInfo; import org.apache.calcite.rel.core.JoinRelType; import org.apache.calcite.rel.logical.LogicalJoin; import org.apache.calcite.rel.rules.CoreRules; @@ -134,20 +137,32 @@ public class FlinkJoinToMultiJoinRule extends RelRule inputNullGenFieldList = new ArrayList<>(); +// Build null generate field list. +buildInputNullGenFieldList(left, right, origJoin.getJoinType(), inputNullGenFieldList); + // Combine the children MultiJoin inputs into an array of inputs for the new MultiJoin. final List projFieldsList = new ArrayList<>(); final List joinFieldRefCountsList = new ArrayList<>(); final List newInputs = -combineInputs(origJoin, left, right, projFieldsList, joinFieldRefCountsList); +combineInputs( +origJoin, +left, +right, +projFieldsList, +joinFieldRefCountsList, +inputNullGenFieldList); // Combine the outer join information from the left and right inputs, and include the outer // join information from the current join, if it's a left/right outer join. final List> joinSpecs = new ArrayList<>(); -combineOuterJoins(origJoin, newInputs, left, right, joinSpecs); +combineOuterJoins(origJoin, newInputs, left, right, joinSpecs, inputNullGenFieldList); // Pull up the join filters from the children MultiJoinRels and combine them with the join // filter associated with this LogicalJoin to form the join filter for the new MultiJoin. -List newJoinFilters = combineJoinFilters(origJoin, left, right); +List newJoinFilters = +combineJoinFilters(origJoin, left, right, inputNullGenFieldList); // Add on the join field reference counts for the join condition associated with this // LogicalJoin. @@ -178,6 +193,77 @@ public class FlinkJoinToMultiJoinRule extends RelRule isNullGenFieldList) { +if (joinType == JoinRelType.INNER) { +buildNullGenFieldList(left, isNullGenFieldList); +buildNullGenFieldList(right, isNullGenFieldList); +} else if (joinType == JoinRelType.LEFT) { +// If origin joinType is left means join fields from right side must be null generated +// fields, so we need only judge these join fields in left side and set null generate +// field is true for all right fields. +buildNullGenFieldList(left, isNullGenFieldList); + +for (int i = 0; i < right.getRowType().getFieldCount(); i++) { +isNullGenFieldList.add(true); +} +} else if (joinType == JoinRelType.RIGHT) { +// If origin joinType is right means join fields from left side must be null generated +// fields, so we need only j
[flink] branch master updated: [FLINK-30270][table-planner] Fix FlinkJoinToMultiJoinRule incorrectly combines Left/Right outer join to MultiJoin error
This is an automated email from the ASF dual-hosted git repository. godfrey 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 7590cb7d847 [FLINK-30270][table-planner] Fix FlinkJoinToMultiJoinRule incorrectly combines Left/Right outer join to MultiJoin error 7590cb7d847 is described below commit 7590cb7d84774b0e8afd7b0af31cc0af762d4c6a Author: zhengyunhong.zyh <337361...@qq.com> AuthorDate: Mon Dec 12 10:49:34 2022 +0800 [FLINK-30270][table-planner] Fix FlinkJoinToMultiJoinRule incorrectly combines Left/Right outer join to MultiJoin error This closes #21487 --- .../rules/logical/FlinkJoinToMultiJoinRule.java| 215 -- .../plan/batch/sql/join/JoinReorderTest.xml| 53 ++-- .../rules/logical/FlinkJoinToMultiJoinRuleTest.xml | 316 + .../logical/FlinkJoinToMultiJoinRuleTest.scala | 91 -- 4 files changed, 475 insertions(+), 200 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRule.java index 838062dc15c..0766c1d4699 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRule.java @@ -18,11 +18,14 @@ package org.apache.flink.table.planner.plan.rules.logical; +import org.apache.flink.table.api.TableException; + import org.apache.calcite.plan.RelOptRuleCall; import org.apache.calcite.plan.RelOptUtil; import org.apache.calcite.plan.RelRule; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.core.Join; +import org.apache.calcite.rel.core.JoinInfo; import org.apache.calcite.rel.core.JoinRelType; import org.apache.calcite.rel.logical.LogicalJoin; import org.apache.calcite.rel.rules.CoreRules; @@ -134,20 +137,32 @@ public class FlinkJoinToMultiJoinRule extends RelRule inputNullGenFieldList = new ArrayList<>(); +// Build null generate field list. +buildInputNullGenFieldList(left, right, origJoin.getJoinType(), inputNullGenFieldList); + // Combine the children MultiJoin inputs into an array of inputs for the new MultiJoin. final List projFieldsList = new ArrayList<>(); final List joinFieldRefCountsList = new ArrayList<>(); final List newInputs = -combineInputs(origJoin, left, right, projFieldsList, joinFieldRefCountsList); +combineInputs( +origJoin, +left, +right, +projFieldsList, +joinFieldRefCountsList, +inputNullGenFieldList); // Combine the outer join information from the left and right inputs, and include the outer // join information from the current join, if it's a left/right outer join. final List> joinSpecs = new ArrayList<>(); -combineOuterJoins(origJoin, newInputs, left, right, joinSpecs); +combineOuterJoins(origJoin, newInputs, left, right, joinSpecs, inputNullGenFieldList); // Pull up the join filters from the children MultiJoinRels and combine them with the join // filter associated with this LogicalJoin to form the join filter for the new MultiJoin. -List newJoinFilters = combineJoinFilters(origJoin, left, right); +List newJoinFilters = +combineJoinFilters(origJoin, left, right, inputNullGenFieldList); // Add on the join field reference counts for the join condition associated with this // LogicalJoin. @@ -178,6 +193,77 @@ public class FlinkJoinToMultiJoinRule extends RelRule isNullGenFieldList) { +if (joinType == JoinRelType.INNER) { +buildNullGenFieldList(left, isNullGenFieldList); +buildNullGenFieldList(right, isNullGenFieldList); +} else if (joinType == JoinRelType.LEFT) { +// If origin joinType is left means join fields from right side must be null generated +// fields, so we need only judge these join fields in left side and set null generate +// field is true for all right fields. +buildNullGenFieldList(left, isNullGenFieldList); + +for (int i = 0; i < right.getRowType().getFieldCount(); i++) { +isNullGenFieldList.add(true); +} +} else if (joinType == JoinRelType.RIGHT) { +// If origin joinType is right means join fields from left side must be null generated +// fields, so we need only judge these joi
[flink] branch master updated: [FLINK-29849][table-planner] Fix event time temporal join on an upsert source may produce incorrect execution plan
This is an automated email from the ASF dual-hosted git repository. godfrey 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 eb44ac01c99 [FLINK-29849][table-planner] Fix event time temporal join on an upsert source may produce incorrect execution plan eb44ac01c99 is described below commit eb44ac01c9969cb22ab832b6b2155b109f015b06 Author: lincoln.lil AuthorDate: Wed Nov 2 16:37:45 2022 +0800 [FLINK-29849][table-planner] Fix event time temporal join on an upsert source may produce incorrect execution plan This closes #21219 --- .../logical/EventTimeTemporalJoinRewriteRule.java | 474 + .../plan/rules/logical/FlinkFilterJoinRule.java| 2 + .../planner/plan/utils/TemporalTableJoinUtil.java | 69 +++ .../logical/FlinkLogicalTableSourceScan.scala | 30 +- .../stream/StreamPhysicalTemporalJoin.scala| 4 +- .../plan/optimize/program/FlinkStreamProgram.scala | 19 +- .../planner/plan/rules/FlinkStreamRuleSets.scala | 2 +- .../TemporalJoinRewriteWithUniqueKeyRule.scala | 7 +- .../stream/StreamPhysicalTableSourceScanRule.scala | 4 +- .../planner/plan/utils/TemporalJoinUtil.scala | 7 +- ...AssignerChangelogNormalizeTransposeRuleTest.xml | 8 +- .../planner/plan/stream/sql/TableScanTest.xml | 8 +- .../plan/stream/sql/join/TemporalJoinTest.xml | 38 ++ ...signerChangelogNormalizeTransposeRuleTest.scala | 2 + .../plan/stream/sql/join/TemporalJoinTest.scala| 64 ++- .../runtime/stream/sql/TemporalJoinITCase.scala| 5 +- .../temporal/TemporalRowTimeJoinOperatorTest.java | 77 17 files changed, 787 insertions(+), 33 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/EventTimeTemporalJoinRewriteRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/EventTimeTemporalJoinRewriteRule.java new file mode 100644 index 000..e5a5f57bb52 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/EventTimeTemporalJoinRewriteRule.java @@ -0,0 +1,474 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.rules.logical; + +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCalc; +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalJoin; +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalRel; +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalSnapshot; +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableSourceScan; +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalWatermarkAssigner; +import org.apache.flink.table.planner.plan.schema.TimeIndicatorRelDataType; +import org.apache.flink.table.planner.plan.utils.TemporalTableJoinUtil; + +import org.apache.flink.shaded.curator5.org.apache.curator.shaded.com.google.common.collect.Lists; + +import org.apache.calcite.plan.RelOptRule; +import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.plan.RelRule; +import org.apache.calcite.plan.hep.HepRelVertex; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.tools.RuleSet; +import org.apache.calcite.tools.RuleSets; + +/** + * Traverses an event time temporal table join {@link RelNode} tree and update the right child to + * set {@link FlinkLogicalTableSourceScan}'s eventTimeSnapshot property to true which will prevent + * it generating a new StreamPhysicalChangelogNormalize later. + * + * the match patterns are as following(8 variants, the three `Calc` nodes are all optional): + * + * {@code + *Join (event time temporal) + * / \ + * RelNode [Calc] + * \ + * Snapshot + *\ + * [Calc] + * \ + *
[flink] branch master updated (fe88dcd8ef5 -> 2851fac9c4c)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from fe88dcd8ef5 [FLINK-30291][Connector/DynamoDB] Update docs to render DynamoDB connector docs add 2851fac9c4c [hotfix][table-planner] Fix tests for FLINK-28988 merged No new revisions were added by this update. Summary of changes: ...termarkAssignerChangelogNormalizeTransposeRuleTest.xml | 15 +++ 1 file changed, 7 insertions(+), 8 deletions(-)
[flink] branch master updated (919f2a0ee2e -> b2203eaef68)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 919f2a0ee2e [hotfix][docs][Connector/Firehose] Add missing references to Firehose connector add b2203eaef68 [FLINK-28988][table-planner] Don't push filters down into the right table for temporal join No new revisions were added by this update. Summary of changes: .../plan/rules/logical/FlinkFilterJoinRule.java| 36 +++ .../physical/stream/ChangelogModeInferenceTest.xml | 108 + .../plan/stream/sql/join/TemporalJoinTest.xml | 64 ++-- .../stream/ChangelogModeInferenceTest.scala| 33 +++ .../runtime/stream/sql/TemporalJoinITCase.scala| 26 - 5 files changed, 228 insertions(+), 39 deletions(-)
[flink] branch release-1.16 updated: [FLINK-29781][table-planner] Fix ChangelogNormalize uses wrong keys after transformation by WatermarkAssignerChangelogNormalizeTransposeRule
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.16 by this push: new 5466716b20d [FLINK-29781][table-planner] Fix ChangelogNormalize uses wrong keys after transformation by WatermarkAssignerChangelogNormalizeTransposeRule 5466716b20d is described below commit 5466716b20d5c720bf29dea560909e7055870555 Author: lincoln.lil AuthorDate: Thu Oct 27 21:24:22 2022 +0800 [FLINK-29781][table-planner] Fix ChangelogNormalize uses wrong keys after transformation by WatermarkAssignerChangelogNormalizeTransposeRule This closes #21225 (cherry picked from commit 5463f244ec69f623d75c15374b55bb8695e92b3e) --- ...arkAssignerChangelogNormalizeTransposeRule.java | 11 ++ .../stream/StreamPhysicalChangelogNormalize.scala | 4 ++ ...AssignerChangelogNormalizeTransposeRuleTest.xml | 43 +- ...signerChangelogNormalizeTransposeRuleTest.scala | 36 ++ 4 files changed, 93 insertions(+), 1 deletion(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/WatermarkAssignerChangelogNormalizeTransposeRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/WatermarkAssignerChangelogNormalizeTransposeRule.java index 1c532d0ca65..276b36389f3 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/WatermarkAssignerChangelogNormalizeTransposeRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/WatermarkAssignerChangelogNormalizeTransposeRule.java @@ -47,6 +47,7 @@ import org.apache.calcite.rex.RexUtil; import org.apache.calcite.util.mapping.Mappings; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -359,6 +360,16 @@ public class WatermarkAssignerChangelogNormalizeTransposeRule inputNode, nodeAndTrait.f1.getTrait( FlinkRelDistributionTraitDef.INSTANCE())); +} else if (currentNode instanceof StreamPhysicalChangelogNormalize) { +final List inputNodeFields = inputNode.getRowType().getFieldNames(); +final List currentNodeFields = currentNode.getRowType().getFieldNames(); +int[] remappedUniqueKeys = +Arrays.stream(((StreamPhysicalChangelogNormalize) currentNode).uniqueKeys()) +.map(ukIdx -> inputNodeFields.indexOf(currentNodeFields.get(ukIdx))) +.toArray(); +currentNode = +((StreamPhysicalChangelogNormalize) currentNode) +.copy(nodeAndTrait.f1, inputNode, remappedUniqueKeys); } else { currentNode = currentNode.copy(nodeAndTrait.f1, Collections.singletonList(inputNode)); diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalChangelogNormalize.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalChangelogNormalize.scala index ac3371af21c..6cfb445ee10 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalChangelogNormalize.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalChangelogNormalize.scala @@ -58,6 +58,10 @@ class StreamPhysicalChangelogNormalize( contextResolvedTable) } + def copy(traitSet: RelTraitSet, input: RelNode, uniqueKeys: Array[Int]): RelNode = { +new StreamPhysicalChangelogNormalize(cluster, traitSet, input, uniqueKeys, contextResolvedTable) + } + override def explainTerms(pw: RelWriter): RelWriter = { val fieldNames = getRowType.getFieldNames super diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/stream/WatermarkAssignerChangelogNormalizeTransposeRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/stream/WatermarkAssignerChangelogNormalizeTransposeRuleTest.xml index b0e1b7e155e..6020b3fd13f 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/stream/WatermarkAssignerChangelogNormalizeTransposeRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/ta
[flink] branch master updated: [FLINK-29781][table-planner] Fix ChangelogNormalize uses wrong keys after transformation by WatermarkAssignerChangelogNormalizeTransposeRule
This is an automated email from the ASF dual-hosted git repository. godfrey 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 5463f244ec6 [FLINK-29781][table-planner] Fix ChangelogNormalize uses wrong keys after transformation by WatermarkAssignerChangelogNormalizeTransposeRule 5463f244ec6 is described below commit 5463f244ec69f623d75c15374b55bb8695e92b3e Author: lincoln.lil AuthorDate: Thu Oct 27 21:24:22 2022 +0800 [FLINK-29781][table-planner] Fix ChangelogNormalize uses wrong keys after transformation by WatermarkAssignerChangelogNormalizeTransposeRule This closes #21225 --- ...arkAssignerChangelogNormalizeTransposeRule.java | 11 ++ .../stream/StreamPhysicalChangelogNormalize.scala | 4 ++ ...AssignerChangelogNormalizeTransposeRuleTest.xml | 43 +- ...signerChangelogNormalizeTransposeRuleTest.scala | 36 ++ 4 files changed, 93 insertions(+), 1 deletion(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/WatermarkAssignerChangelogNormalizeTransposeRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/WatermarkAssignerChangelogNormalizeTransposeRule.java index 1c532d0ca65..276b36389f3 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/WatermarkAssignerChangelogNormalizeTransposeRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/WatermarkAssignerChangelogNormalizeTransposeRule.java @@ -47,6 +47,7 @@ import org.apache.calcite.rex.RexUtil; import org.apache.calcite.util.mapping.Mappings; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -359,6 +360,16 @@ public class WatermarkAssignerChangelogNormalizeTransposeRule inputNode, nodeAndTrait.f1.getTrait( FlinkRelDistributionTraitDef.INSTANCE())); +} else if (currentNode instanceof StreamPhysicalChangelogNormalize) { +final List inputNodeFields = inputNode.getRowType().getFieldNames(); +final List currentNodeFields = currentNode.getRowType().getFieldNames(); +int[] remappedUniqueKeys = +Arrays.stream(((StreamPhysicalChangelogNormalize) currentNode).uniqueKeys()) +.map(ukIdx -> inputNodeFields.indexOf(currentNodeFields.get(ukIdx))) +.toArray(); +currentNode = +((StreamPhysicalChangelogNormalize) currentNode) +.copy(nodeAndTrait.f1, inputNode, remappedUniqueKeys); } else { currentNode = currentNode.copy(nodeAndTrait.f1, Collections.singletonList(inputNode)); diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalChangelogNormalize.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalChangelogNormalize.scala index ac3371af21c..6cfb445ee10 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalChangelogNormalize.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalChangelogNormalize.scala @@ -58,6 +58,10 @@ class StreamPhysicalChangelogNormalize( contextResolvedTable) } + def copy(traitSet: RelTraitSet, input: RelNode, uniqueKeys: Array[Int]): RelNode = { +new StreamPhysicalChangelogNormalize(cluster, traitSet, input, uniqueKeys, contextResolvedTable) + } + override def explainTerms(pw: RelWriter): RelWriter = { val fieldNames = getRowType.getFieldNames super diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/stream/WatermarkAssignerChangelogNormalizeTransposeRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/stream/WatermarkAssignerChangelogNormalizeTransposeRuleTest.xml index b0e1b7e155e..6020b3fd13f 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/stream/WatermarkAssignerChangelogNormalizeTransposeRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/stream/WatermarkAssignerChangelogNormalizeTransposeRuleTest.
[flink-web] branch asf-site updated (7a0e58e12 -> 32c6bc7d5)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch asf-site in repository https://gitbox.apache.org/repos/asf/flink-web.git from 7a0e58e12 Rebuild the website add b13b11041 [hotfix] Fix some typos in 1.16 announcement add 32c6bc7d5 Rebuild the website No new revisions were added by this update. Summary of changes: _posts/2022-10-28-1.16-announcement.md | 9 + content/blog/feed.xml | 9 + content/blog/index.html| 4 ++-- content/index.html | 4 ++-- content/news/2022/10/28/1.16-announcement.html | 9 + content/zh/index.html | 4 ++-- 6 files changed, 21 insertions(+), 18 deletions(-)
[flink-web] branch asf-site updated: Announcement blogpost for the 1.16 release
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch asf-site in repository https://gitbox.apache.org/repos/asf/flink-web.git The following commit(s) were added to refs/heads/asf-site by this push: new 1efbf9b6b Announcement blogpost for the 1.16 release 1efbf9b6b is described below commit 1efbf9b6b9a7827632783073c8706751cdfdcff3 Author: godfreyhe AuthorDate: Fri Sep 30 14:50:48 2022 +0800 Announcement blogpost for the 1.16 release --- _config.yml| 45 ++-- _posts/2022-10-28-1.16-announcement.md | 403 + content/q/gradle-quickstart.sh | 2 +- content/q/sbt-quickstart.sh| 2 +- img/rocksdb_rescaling_benchmark.png| Bin 0 -> 450221 bytes q/gradle-quickstart.sh | 2 +- q/quickstart-scala.sh | 2 +- q/quickstart.sh| 2 +- q/sbt-quickstart.sh| 2 +- 9 files changed, 431 insertions(+), 29 deletions(-) diff --git a/_config.yml b/_config.yml index 57cbc3441..a971ce731 100644 --- a/_config.yml +++ b/_config.yml @@ -9,8 +9,8 @@ url: https://flink.apache.org DOCS_BASE_URL: https://nightlies.apache.org/flink/ -FLINK_VERSION_STABLE: 1.15.2 -FLINK_VERSION_STABLE_SHORT: "1.15" +FLINK_VERSION_STABLE: 1.16.0 +FLINK_VERSION_STABLE_SHORT: "1.16" FLINK_ISSUES_URL: https://issues.apache.org/jira/browse/FLINK FLINK_GITHUB_URL: https://github.com/apache/flink @@ -73,6 +73,22 @@ FLINK_TABLE_STORE_GITHUB_REPO_NAME: flink-table-store # md1_url: https://repo.maven.apache.org/maven2/org/apache/flink/flink-metrics-prometheus_2.12/1.7.1/flink-metrics-prometheus_2.12-1.7.1.jar.sha1 flink_releases: + - version_short: "1.16" +binary_release: + name: "Apache Flink 1.16.0" + scala_212: +id: "1160-download_212" +url: "https://www.apache.org/dyn/closer.lua/flink/flink-1.16.0/flink-1.16.0-bin-scala_2.12.tgz; +asc_url: "https://downloads.apache.org/flink/flink-1.16.0/flink-1.16.0-bin-scala_2.12.tgz.asc; +sha512_url: "https://downloads.apache.org/flink/flink-1.16.0/flink-1.16.0-bin-scala_2.12.tgz.sha512; +source_release: + name: "Apache Flink 1.16.0" + id: "1160-download-source" + url: "https://www.apache.org/dyn/closer.lua/flink/flink-1.16.0/flink-1.16.0-src.tgz; + asc_url: "https://downloads.apache.org/flink/flink-1.16.0/flink-1.16.0-src.tgz.asc; + sha512_url: "https://downloads.apache.org/flink/flink-1.16.0/flink-1.16.0-src.tgz.sha512; +release_notes_url: "https://nightlies.apache.org/flink/flink-docs-release-1.16/release-notes/flink-1.16; + - version_short: "1.15" binary_release: name: "Apache Flink 1.15.2" @@ -88,27 +104,6 @@ flink_releases: asc_url: "https://downloads.apache.org/flink/flink-1.15.2/flink-1.15.2-src.tgz.asc; sha512_url: "https://downloads.apache.org/flink/flink-1.15.2/flink-1.15.2-src.tgz.sha512; release_notes_url: "https://nightlies.apache.org/flink/flink-docs-release-1.15/release-notes/flink-1.15; - - -version_short: "1.14" -binary_release: - name: "Apache Flink 1.14.6" - scala_211: -id: "1146-download_211" -url: "https://www.apache.org/dyn/closer.lua/flink/flink-1.14.6/flink-1.14.6-bin-scala_2.11.tgz; -asc_url: "https://downloads.apache.org/flink/flink-1.14.6/flink-1.14.6-bin-scala_2.11.tgz.asc; -sha512_url: "https://downloads.apache.org/flink/flink-1.14.6/flink-1.14.6-bin-scala_2.11.tgz.sha512; - scala_212: -id: "1146-download_212" -url: "https://www.apache.org/dyn/closer.lua/flink/flink-1.14.6/flink-1.14.6-bin-scala_2.12.tgz; -asc_url: "https://downloads.apache.org/flink/flink-1.14.6/flink-1.14.6-bin-scala_2.12.tgz.asc; -sha512_url: "https://downloads.apache.org/flink/flink-1.14.6/flink-1.14.6-bin-scala_2.12.tgz.sha512; -source_release: - name: "Apache Flink 1.14.6" - id: "1146-download-source" - url: "https://www.apache.org/dyn/closer.lua/flink/flink-1.14.6/flink-1.14.6-src.tgz; - asc_url: "https://downloads.apache.org/flink/flink-1.14.6/flink-1.14.6-src.tgz.asc; - sha512_url: "https://downloads.apache.org/flink/flink-1.14.6/flink-1.14.6-src.tgz.sha512; -release_notes_url: "https://nightlies.apache.org/flink/flink-docs-release-1.14/release-notes/flink-1.14; flink_statefun_releases: - @@ -244,6 +239,10 @@ component_releases: release_archive: flink: + - +version_short: "1.16" +version_long: 1.16.0 +release_date: 2022-10-28 - version_short: "1.15" version_long: 1.15.2 diff --
[flink] branch release-1.16 updated: [FLINK-29126][hive] Fix splitting file optimization doesn't work for orc format
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.16 by this push: new cbc9d462b29 [FLINK-29126][hive] Fix splitting file optimization doesn't work for orc format cbc9d462b29 is described below commit cbc9d462b295243a61ebc544d9cf9ff6fa2a8aa6 Author: luoyuxia AuthorDate: Mon Aug 29 11:07:39 2022 +0800 [FLINK-29126][hive] Fix splitting file optimization doesn't work for orc format This closes #20694 (cherry picked from commit cf70844a56a0994dfcd7fb1859408683f2b621a3) --- .../docs/connectors/table/hive/hive_read_write.md | 7 +- .../docs/connectors/table/hive/hive_read_write.md | 9 ++- .../apache/flink/connectors/hive/HiveOptions.java | 6 ++ .../flink/connectors/hive/HiveSourceBuilder.java | 20 - .../connectors/hive/HiveSourceFileEnumerator.java | 88 +- .../hive/HiveSourceFileEnumeratorTest.java | 44 ++- .../connectors/hive/PartitionMonitorTest.java | 6 +- 7 files changed, 154 insertions(+), 26 deletions(-) diff --git a/docs/content.zh/docs/connectors/table/hive/hive_read_write.md b/docs/content.zh/docs/connectors/table/hive/hive_read_write.md index 1bcb8ec4b94..033e8bb1486 100644 --- a/docs/content.zh/docs/connectors/table/hive/hive_read_write.md +++ b/docs/content.zh/docs/connectors/table/hive/hive_read_write.md @@ -183,7 +183,12 @@ Flink 允许你灵活的配置并发推断策略。你可以在 `TableConfig` -**注意:** 目前上述参数仅适用于 ORC 格式的 Hive 表。 +{{< hint warning >}} +**注意:** +- 为了调整数据分片的大小, Flink 首先将计算得到所有分区下的所有文件的大小。 + 但是这在分区数量很多的情况下会比较耗时,你可以配置作业参数 `table.exec.hive.calculate-partition-size.thread-num`(默认为3)为一个更大的值使用更多的线程来进行加速。 +- 目前上述参数仅适用于 ORC 格式的 Hive 表。 +{{< /hint >}} ### 加载分区切片 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 95f377732d6..3636eaf599c 100644 --- a/docs/content/docs/connectors/table/hive/hive_read_write.md +++ b/docs/content/docs/connectors/table/hive/hive_read_write.md @@ -198,8 +198,13 @@ Users can do some performance tuning by tuning the split's size with the follow - -**NOTE**: Currently, these two configurations only works for the Hive table stored as ORC format. +{{< hint warning >}} +**NOTE**: +- To tune the split's size, Flink will first get all files' size for all partitions. + If there are too many partitions, it maybe time-consuming, + then you can configure the job configuration `table.exec.hive.calculate-partition-size.thread-num` (3 by default) to a bigger value to enable more threads to speed up the process. +- Currently, these configurations for tuning split size only works for the Hive table stored as ORC format. +{{< /hint >}} ### Load Partition Splits diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveOptions.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveOptions.java index 38057a6e07a..9d7ce987578 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveOptions.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveOptions.java @@ -96,6 +96,12 @@ public class HiveOptions { + " When the value is over estimated, Flink will tend to pack Hive's data into less splits, which will be helpful when Hive's table contains many small files." + " And vice versa. It only works for the Hive table stored as ORC format."); +public static final ConfigOption TABLE_EXEC_HIVE_CALCULATE_PARTITION_SIZE_THREAD_NUM = +key("table.exec.hive.calculate-partition-size.thread-num") +.intType() +.defaultValue(3) +.withDeprecatedKeys("The thread number to calculate partition's size."); + public static final ConfigOption TABLE_EXEC_HIVE_DYNAMIC_GROUPING_ENABLED = key("table.exec.hive.sink.sort-by-dynamic-partition.enable") .booleanType() diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceBuilder.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceBuilder.java index 3837d718817..bb6ee5a6c1e 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceBuilder.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceBuilder.java @@ -296,15 +296,18 @@ public class HiveSourceBuilder { } private void setFlinkConfigurationToJobConf(
[flink] branch master updated (2a4da2501eb -> cf70844a56a)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 2a4da2501eb [hotfix][test] Refactor test to use correct syntax for JUnit5 annotation add cf70844a56a [FLINK-29126][hive] Fix splitting file optimization doesn't work for orc format No new revisions were added by this update. Summary of changes: .../docs/connectors/table/hive/hive_read_write.md | 7 +- .../docs/connectors/table/hive/hive_read_write.md | 9 ++- .../apache/flink/connectors/hive/HiveOptions.java | 6 ++ .../flink/connectors/hive/HiveSourceBuilder.java | 20 - .../connectors/hive/HiveSourceFileEnumerator.java | 88 +- .../hive/HiveSourceFileEnumeratorTest.java | 44 ++- .../connectors/hive/PartitionMonitorTest.java | 6 +- 7 files changed, 154 insertions(+), 26 deletions(-)
[flink] branch release-1.16 updated: [FLINK-29280][table-planner] Fix join hints could not be propagated in subquery
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.16 by this push: new b37a8153f22 [FLINK-29280][table-planner] Fix join hints could not be propagated in subquery b37a8153f22 is described below commit b37a8153f22b62982ca144604a34056246f6f36c Author: xuyang AuthorDate: Tue Sep 13 18:59:43 2022 +0800 [FLINK-29280][table-planner] Fix join hints could not be propagated in subquery This closes #20823 (cherry picked from commit 22cb554008320e6684280b5205f93d7a6f685c6c) --- .../apache/calcite/sql2rel/RelDecorrelator.java| 4 + .../plan/rules/logical/SubQueryDecorrelator.java | 22 +- .../table/planner/calcite/FlinkPlannerImpl.scala | 3 +- .../rules/logical/FlinkSubQueryRemoveRule.scala| 13 +- .../planner/plan/hints/batch/JoinHintTestBase.java | 40 +++ .../plan/hints/batch/BroadcastJoinHintTest.xml | 275 .../plan/hints/batch/NestLoopJoinHintTest.xml | 275 .../plan/hints/batch/ShuffleHashJoinHintTest.xml | 276 + .../plan/hints/batch/ShuffleMergeJoinHintTest.xml | 276 + .../optimize/ClearQueryBlockAliasResolverTest.xml | 152 +--- .../planner/plan/optimize/JoinHintResolverTest.xml | 156 +--- 11 files changed, 1215 insertions(+), 277 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java index 0d132041278..cf2d4de2c65 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java @@ -17,6 +17,7 @@ package org.apache.calcite.sql2rel; import org.apache.flink.table.planner.alias.ClearJoinHintWithInvalidPropagationShuttle; +import org.apache.flink.table.planner.hint.FlinkHints; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -207,6 +208,9 @@ public class RelDecorrelator implements ReflectiveVisitor { // - FLINK MODIFICATION BEGIN - +// replace all join hints with upper case +newRootRel = FlinkHints.capitalizeJoinHints(newRootRel); + // clear join hints which are propagated into wrong query block // The hint QueryBlockAlias will be added when building a RelNode tree before. It is used to // distinguish the query block in the SQL. diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/SubQueryDecorrelator.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/SubQueryDecorrelator.java index 828286b5fc6..6a27f45839e 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/SubQueryDecorrelator.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/SubQueryDecorrelator.java @@ -35,6 +35,7 @@ import org.apache.calcite.rel.core.Project; import org.apache.calcite.rel.core.SetOp; import org.apache.calcite.rel.core.Sort; import org.apache.calcite.rel.core.Values; +import org.apache.calcite.rel.hint.Hintable; import org.apache.calcite.rel.logical.LogicalAggregate; import org.apache.calcite.rel.logical.LogicalCorrelate; import org.apache.calcite.rel.logical.LogicalFilter; @@ -492,6 +493,7 @@ public class SubQueryDecorrelator extends RelShuttleImpl { } } RelNode newProject = RelOptUtil.createProject(newInput, projects, false); +newProject = ((LogicalProject) newProject).withHints(rel.getHints()); final RexNode newCorCondition; if (frame.c != null) { @@ -544,11 +546,13 @@ public class SubQueryDecorrelator extends RelShuttleImpl { // Using LogicalFilter.create instead of RelBuilder.filter to create Filter // because RelBuilder.filter method does not have VariablesSet arg. -final LogicalFilter newFilter = +final RelNode newFilter = LogicalFilter.create( -frame.r, -remainingCondition, - com.google.common.collect.ImmutableSet.copyOf(rel.getVariablesSet())); +frame.r, +remainingCondition, + com.google.common.collect.ImmutableSet.copyOf( +rel.getVariablesSet())) +.withHints(rel.getHints()); // Adds input's correlation
[flink] branch master updated (a02b2c232ea -> 22cb5540083)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from a02b2c232ea [FLINK-28738][table-planner] Adds a user doc about the determinism in streaming add 22cb5540083 [FLINK-29280][table-planner] Fix join hints could not be propagated in subquery No new revisions were added by this update. Summary of changes: .../apache/calcite/sql2rel/RelDecorrelator.java| 4 + .../plan/rules/logical/SubQueryDecorrelator.java | 22 +- .../table/planner/calcite/FlinkPlannerImpl.scala | 3 +- .../rules/logical/FlinkSubQueryRemoveRule.scala| 13 +- .../planner/plan/hints/batch/JoinHintTestBase.java | 40 +++ .../plan/hints/batch/BroadcastJoinHintTest.xml | 275 .../plan/hints/batch/NestLoopJoinHintTest.xml | 275 .../plan/hints/batch/ShuffleHashJoinHintTest.xml | 276 + .../plan/hints/batch/ShuffleMergeJoinHintTest.xml | 276 + .../optimize/ClearQueryBlockAliasResolverTest.xml | 152 +--- .../planner/plan/optimize/JoinHintResolverTest.xml | 156 +--- 11 files changed, 1215 insertions(+), 277 deletions(-)
[flink] branch release-1.16 updated: [FLINK-28738][table-planner] Adds a user doc about the determinism in streaming
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.16 by this push: new be73c9695d0 [FLINK-28738][table-planner] Adds a user doc about the determinism in streaming be73c9695d0 is described below commit be73c9695d01e3d3164b6c89342a1e41fa4ea450 Author: lincoln lee AuthorDate: Mon Aug 22 17:01:03 2022 +0800 [FLINK-28738][table-planner] Adds a user doc about the determinism in streaming This closes #20679 (cherry picked from commit a02b2c232ea2fb1b22bd0e5c290e4c6f0217549b) --- .../docs/dev/table/concepts/determinism.md | 261 .../content.zh/docs/dev/table/concepts/overview.md | 3 +- .../content/docs/dev/table/concepts/determinism.md | 269 + docs/content/docs/dev/table/concepts/overview.md | 1 + 4 files changed, 533 insertions(+), 1 deletion(-) diff --git a/docs/content.zh/docs/dev/table/concepts/determinism.md b/docs/content.zh/docs/dev/table/concepts/determinism.md new file mode 100644 index 000..7cee29a4773 --- /dev/null +++ b/docs/content.zh/docs/dev/table/concepts/determinism.md @@ -0,0 +1,261 @@ +--- +title: "流上的确定性 (Determinism in Continuous Queries)" +weight: 2 +type: docs +aliases: + - /zh/dev/table/streaming/determinism.html +--- + + +# 流上的确定性(Determinism In Continuous Queries) +本文主要介绍了以下内容: + +1. 什么是确定性? +2. 批处理都是确定性的吗? + - 两个非确定性结果的批查询示例 + - 批处理中的不确定性因素 +3. 流上的确定性 +- 流上的不确定性 +- 流上的不确定更新 +- 如何消除流查询的不确定性影响 + +## 1. 什么是确定性? +引用 SQL 标准中对确定性的描述:"如果一个操作在重复相同的输入值时能保证计算出相同的结果,那么该操作就是确定性的"。 + +## 2. 批处理都是确定性的吗? +在经典的批处理场景,对于给定的有界数据集,重复执行同一查询会得到一致的结果,这是对确定性最直观的理解。但实际上,同一个查询在批处理上也并不总是能得到一致的结果,来看两个查询示例: + +### 2.1 两个非确定性结果的批查询示例 +比如有一张新建的网站点击日志表: +```sql +CREATE TABLE clicks ( +uid VARCHAR(128), +cTime TIMESTAMP(3), +url VARCHAR(256) +) +``` + +新写入了一些数据: +``` ++--+-++ +| uid | cTime |url | ++--+-++ +| Mary | 2022-08-22 12:00:01 | /home | +| Bob | 2022-08-22 12:00:01 | /home | +| Mary | 2022-08-22 12:00:05 | /prod?id=1 | +| Liz | 2022-08-22 12:01:00 | /home | +| Mary | 2022-08-22 12:01:30 | /cart | +| Bob | 2022-08-22 12:01:35 | /prod?id=3 | ++--+-++ +``` + +1. 查询 1 对日志表进行了时间过滤,希望筛选出最近 2 分钟的点击日志: +```sql +SELECT * FROM clicks +WHERE cTime BETWEEN TIMESTAMPADD(MINUTE, -2, CURRENT_TIMESTAMP) AND CURRENT_TIMESTAMP; +``` +在 '2022-08-22 12:02:00' 时刻提交该查询时,查询返回了表中全部的 6 条数据, 一分钟后,也就是在 '2022-08-22 12:03:00' 时刻再次执行该查询时, 只返回了 3 条数据: +``` ++--+-++ +| uid | cTime |url | ++--+-++ +| Liz | 2022-08-22 12:01:00 | /home | +| Mary | 2022-08-22 12:01:30 | /cart | +| Bob | 2022-08-22 12:01:35 | /prod?id=3 | ++--+-++ +``` + +2. 查询 2 希望对每条返回记录添加一个唯一标识(因为 clicks 表没有定义主键): +```sql +SELECT UUID() AS uuid, * FROM clicks LIMIT 3; +``` +连续执行两次该查询,每条记录都生成了不同的 `uuid` 标识: +``` +-- 第 1 次执行 +++--+-++ +| uuid | uid | cTime |url | +++--+-++ +| 4894-16d4-44d0-a763-03f... | Mary | 2022-08-22 12:00:01 | /home | +| ed26fd46-960e-4228-aaf2-0aa... | Bob | 2022-08-22 12:00:01 | /home | +| 1886afc7-dfc6-4b20-a881-b0e... | Mary | 2022-08-22 12:00:05 | /prod?id=1 | +++--+-++ + +-- 第 2 次执行 +++--+-++ +| uuid | uid | cTime |url | +++--+-++ +| 95f7301f-bcf2-4b6f-9cf3-1ea... | Mary | 2022-08-22 12:00:01 | /home | +| 63301e2d-d180-4089-876f-683... | Bob | 2022-08-22 12:00:01 | /home | +| f24456d3-e942-43d1-a00f-fdb... | Mary | 2022-08-22 12:00:05 | /prod?id=1 | +++--+-++ +``` + +### 2.2 批处理中的不确定性因素 +批处理中的不确定性因素, 主要是由不确定函数造成的,上述两个查询示例中,内置函数 `CURRENT_TIMESTAMP` 和 `UUID()` +在批处理中的行为是有差异的,继续看一个查询示例: +```sql +SELECT CURRENT_TIMESTAMP, * FROM clicks; +``` + +`CURRENT_TIMESTAMP` 在返回的记录上都是同一个值: +``` ++-+--+-++ +| CURRENT_TIMESTAMP | uid | cTime |url | ++-+--+-++ +| 2022-08-23 17:25:46.831 | Mary | 2022-08-22 12:00:01 | /home | +| 2022-08-23 17:25:46.831 | Bob | 2022-08-22 12:00:01 | /home | +| 2022-08-23 17:25:46.831 | Mary | 2022-08-22 12:00:05 | /prod?id=1 | +| 202
[flink] branch master updated: [FLINK-28738][table-planner] Adds a user doc about the determinism in streaming
This is an automated email from the ASF dual-hosted git repository. godfrey 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 a02b2c232ea [FLINK-28738][table-planner] Adds a user doc about the determinism in streaming a02b2c232ea is described below commit a02b2c232ea2fb1b22bd0e5c290e4c6f0217549b Author: lincoln lee AuthorDate: Mon Aug 22 17:01:03 2022 +0800 [FLINK-28738][table-planner] Adds a user doc about the determinism in streaming This closes #20679 --- .../docs/dev/table/concepts/determinism.md | 261 .../content.zh/docs/dev/table/concepts/overview.md | 3 +- .../content/docs/dev/table/concepts/determinism.md | 269 + docs/content/docs/dev/table/concepts/overview.md | 1 + 4 files changed, 533 insertions(+), 1 deletion(-) diff --git a/docs/content.zh/docs/dev/table/concepts/determinism.md b/docs/content.zh/docs/dev/table/concepts/determinism.md new file mode 100644 index 000..7cee29a4773 --- /dev/null +++ b/docs/content.zh/docs/dev/table/concepts/determinism.md @@ -0,0 +1,261 @@ +--- +title: "流上的确定性 (Determinism in Continuous Queries)" +weight: 2 +type: docs +aliases: + - /zh/dev/table/streaming/determinism.html +--- + + +# 流上的确定性(Determinism In Continuous Queries) +本文主要介绍了以下内容: + +1. 什么是确定性? +2. 批处理都是确定性的吗? + - 两个非确定性结果的批查询示例 + - 批处理中的不确定性因素 +3. 流上的确定性 +- 流上的不确定性 +- 流上的不确定更新 +- 如何消除流查询的不确定性影响 + +## 1. 什么是确定性? +引用 SQL 标准中对确定性的描述:"如果一个操作在重复相同的输入值时能保证计算出相同的结果,那么该操作就是确定性的"。 + +## 2. 批处理都是确定性的吗? +在经典的批处理场景,对于给定的有界数据集,重复执行同一查询会得到一致的结果,这是对确定性最直观的理解。但实际上,同一个查询在批处理上也并不总是能得到一致的结果,来看两个查询示例: + +### 2.1 两个非确定性结果的批查询示例 +比如有一张新建的网站点击日志表: +```sql +CREATE TABLE clicks ( +uid VARCHAR(128), +cTime TIMESTAMP(3), +url VARCHAR(256) +) +``` + +新写入了一些数据: +``` ++--+-++ +| uid | cTime |url | ++--+-++ +| Mary | 2022-08-22 12:00:01 | /home | +| Bob | 2022-08-22 12:00:01 | /home | +| Mary | 2022-08-22 12:00:05 | /prod?id=1 | +| Liz | 2022-08-22 12:01:00 | /home | +| Mary | 2022-08-22 12:01:30 | /cart | +| Bob | 2022-08-22 12:01:35 | /prod?id=3 | ++--+-++ +``` + +1. 查询 1 对日志表进行了时间过滤,希望筛选出最近 2 分钟的点击日志: +```sql +SELECT * FROM clicks +WHERE cTime BETWEEN TIMESTAMPADD(MINUTE, -2, CURRENT_TIMESTAMP) AND CURRENT_TIMESTAMP; +``` +在 '2022-08-22 12:02:00' 时刻提交该查询时,查询返回了表中全部的 6 条数据, 一分钟后,也就是在 '2022-08-22 12:03:00' 时刻再次执行该查询时, 只返回了 3 条数据: +``` ++--+-++ +| uid | cTime |url | ++--+-++ +| Liz | 2022-08-22 12:01:00 | /home | +| Mary | 2022-08-22 12:01:30 | /cart | +| Bob | 2022-08-22 12:01:35 | /prod?id=3 | ++--+-++ +``` + +2. 查询 2 希望对每条返回记录添加一个唯一标识(因为 clicks 表没有定义主键): +```sql +SELECT UUID() AS uuid, * FROM clicks LIMIT 3; +``` +连续执行两次该查询,每条记录都生成了不同的 `uuid` 标识: +``` +-- 第 1 次执行 +++--+-++ +| uuid | uid | cTime |url | +++--+-++ +| 4894-16d4-44d0-a763-03f... | Mary | 2022-08-22 12:00:01 | /home | +| ed26fd46-960e-4228-aaf2-0aa... | Bob | 2022-08-22 12:00:01 | /home | +| 1886afc7-dfc6-4b20-a881-b0e... | Mary | 2022-08-22 12:00:05 | /prod?id=1 | +++--+-++ + +-- 第 2 次执行 +++--+-++ +| uuid | uid | cTime |url | +++--+-++ +| 95f7301f-bcf2-4b6f-9cf3-1ea... | Mary | 2022-08-22 12:00:01 | /home | +| 63301e2d-d180-4089-876f-683... | Bob | 2022-08-22 12:00:01 | /home | +| f24456d3-e942-43d1-a00f-fdb... | Mary | 2022-08-22 12:00:05 | /prod?id=1 | +++--+-++ +``` + +### 2.2 批处理中的不确定性因素 +批处理中的不确定性因素, 主要是由不确定函数造成的,上述两个查询示例中,内置函数 `CURRENT_TIMESTAMP` 和 `UUID()` +在批处理中的行为是有差异的,继续看一个查询示例: +```sql +SELECT CURRENT_TIMESTAMP, * FROM clicks; +``` + +`CURRENT_TIMESTAMP` 在返回的记录上都是同一个值: +``` ++-+--+-++ +| CURRENT_TIMESTAMP | uid | cTime |url | ++-+--+-++ +| 2022-08-23 17:25:46.831 | Mary | 2022-08-22 12:00:01 | /home | +| 2022-08-23 17:25:46.831 | Bob | 2022-08-22 12:00:01 | /home | +| 2022-08-23 17:25:46.831 | Mary | 2022-08-22 12:00:05 | /prod?id=1 | +| 2022-08-23 17:25:46.831 | Liz | 2022-08-22 12:01:00 | /home | +| 2022-08-23 17:25:46.
[flink] branch release-1.16 updated (66f996aecaf -> 390612320fa)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git from 66f996aecaf [FLINK-29287][tests] Rewrite sql-client packaging tests new 2f89d63d57b [FLINK-28569][table-planner] Add projectRowType to RowTypeUtils and deprecate AggCodeGenHelper#projectRowType new bab5c93ca1f [FLINK-28569][table-planner] Move non-deterministic test functions to userDefinedScalarFunctions new 390612320fa [FLINK-28569][table-planner] Fix SinkUpsertMaterializer that should be aware of the input upsertKey if it is not empty to prevent wrong results 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: .../plan/nodes/exec/batch/BatchExecSink.java | 3 +- .../plan/nodes/exec/common/CommonExecSink.java | 25 +++- .../plan/nodes/exec/stream/StreamExecSink.java | 12 +- .../table/planner/plan/utils/UpsertKeyUtil.java| 66 + .../table/planner/typeutils/RowTypeUtils.java | 35 + .../codegen/agg/batch/AggCodeGenHelper.scala | 4 - .../codegen/agg/batch/HashAggCodeGenerator.scala | 3 +- .../codegen/agg/batch/SortAggCodeGenerator.scala | 3 +- .../codegen/agg/batch/WindowCodeGenerator.scala| 3 +- .../nodes/physical/stream/StreamPhysicalSink.scala | 10 +- .../nodes/exec/stream/TableSinkJsonPlanTest.java | 36 + .../planner/plan/utils/UpsertKeyUtilTest.java | 52 .../utils/JavaUserDefinedScalarFunctions.java | 4 + .../table/planner/typeutils/RowTypeUtilsTest.java | 46 +++ .../testChangelogSource.out| 1 + .../testUpsertSource.out | 1 + .../testDeduplication.out | 1 + .../ExpandJsonPlanTest_jsonplan/testExpand.out | 1 + ...tDistinctAggCalls[isMiniBatchEnabled=false].out | 1 + ...stDistinctAggCalls[isMiniBatchEnabled=true].out | 1 + ...gCallsWithGroupBy[isMiniBatchEnabled=false].out | 1 + ...ggCallsWithGroupBy[isMiniBatchEnabled=true].out | 1 + ...erDefinedAggCalls[isMiniBatchEnabled=false].out | 1 + ...serDefinedAggCalls[isMiniBatchEnabled=true].out | 1 + .../testEventTimeTumbleWindow.out | 1 + .../testProcTimeTumbleWindow.out | 1 + .../testIncrementalAggregate.out | 1 + ...lAggregateWithSumCountDistinctAndRetraction.out | 1 + .../testInnerJoinWithEqualPk.out | 1 + ...WithNonDeterministicFuncSinkWithDifferentPk.out | 147 + .../utils/userDefinedScalarFunctions.scala | 53 +++- .../plan/stream/sql/NonDeterministicDagTest.scala | 64 + .../runtime/stream/sql/TableSinkITCase.scala | 62 + .../operators/sink/SinkUpsertMaterializer.java | 125 +- .../operators/sink/SinkUpsertMaterializerTest.java | 102 +++--- 35 files changed, 746 insertions(+), 124 deletions(-) create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/UpsertKeyUtil.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/UpsertKeyUtilTest.java create mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testCdcWithNonDeterministicFuncSinkWithDifferentPk.out
[flink] 01/03: [FLINK-28569][table-planner] Add projectRowType to RowTypeUtils and deprecate AggCodeGenHelper#projectRowType
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git commit 2f89d63d57b19d16b4fbfd0bb5b70939a511244e Author: lincoln.lil AuthorDate: Thu Sep 8 18:08:58 2022 +0800 [FLINK-28569][table-planner] Add projectRowType to RowTypeUtils and deprecate AggCodeGenHelper#projectRowType This closes #20791 (cherry picked from commit dbcd2d7b86fcb7fa7a26e181f1719ea4c6dad828) --- .../table/planner/typeutils/RowTypeUtils.java | 35 .../codegen/agg/batch/AggCodeGenHelper.scala | 4 -- .../codegen/agg/batch/HashAggCodeGenerator.scala | 3 +- .../codegen/agg/batch/SortAggCodeGenerator.scala | 3 +- .../codegen/agg/batch/WindowCodeGenerator.scala| 3 +- .../table/planner/typeutils/RowTypeUtilsTest.java | 46 ++ 6 files changed, 87 insertions(+), 7 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/typeutils/RowTypeUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/typeutils/RowTypeUtils.java index ffb9a68a131..4d9879d7b99 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/typeutils/RowTypeUtils.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/typeutils/RowTypeUtils.java @@ -18,7 +18,13 @@ package org.apache.flink.table.planner.typeutils; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; + +import javax.annotation.Nonnull; + import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -46,4 +52,33 @@ public class RowTypeUtils { } return result; } + +/** + * Returns projected {@link RowType} by given projection indexes over original {@link RowType}. + * Will raise an error when projection index beyond the field count of original rowType. + * + * @param rowType source row type + * @param projection indexes array + * @return projected {@link RowType} + */ +public static RowType projectRowType(@Nonnull RowType rowType, @Nonnull int[] projection) +throws IllegalArgumentException { +final int fieldCnt = rowType.getFieldCount(); +return RowType.of( +Arrays.stream(projection) +.mapToObj( +index -> { +if (index >= fieldCnt) { +throw new IllegalArgumentException( +String.format( +"Invalid projection index: %d of source rowType size: %d", +index, fieldCnt)); +} +return rowType.getTypeAt(index); +}) +.toArray(LogicalType[]::new), +Arrays.stream(projection) +.mapToObj(index -> rowType.getFieldNames().get(index)) +.toArray(String[]::new)); +} } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/AggCodeGenHelper.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/AggCodeGenHelper.scala index bb1135fc236..c401e50ea24 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/AggCodeGenHelper.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/AggCodeGenHelper.scala @@ -93,10 +93,6 @@ object AggCodeGenHelper { .asInstanceOf[Map[AggregateFunction[_, _], String]] } - def projectRowType(rowType: RowType, mapping: Array[Int]): RowType = { -RowType.of(mapping.map(rowType.getTypeAt), mapping.map(rowType.getFieldNames.get(_))) - } - /** Add agg handler to class member and open it. */ private[flink] def addAggsHandler( aggsHandler: GeneratedAggsHandleFunction, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/HashAggCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/HashAggCodeGenerator.scala index 550a93df3fb..c768ffd5f70 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/HashAggCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/HashAggCodeGenerator.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.functions.Aggreg
[flink] 02/03: [FLINK-28569][table-planner] Move non-deterministic test functions to userDefinedScalarFunctions
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git commit bab5c93ca1f839a8e3a12f0149bb9b4d0ad7f83f Author: lincoln.lil AuthorDate: Thu Sep 8 18:11:15 2022 +0800 [FLINK-28569][table-planner] Move non-deterministic test functions to userDefinedScalarFunctions This closes #20791 (cherry picked from commit 7df115c016cab326627e4c12b2a4c449c1794e95) --- .../utils/userDefinedScalarFunctions.scala | 53 +- .../plan/stream/sql/NonDeterministicDagTest.scala | 64 ++ 2 files changed, 58 insertions(+), 59 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/utils/userDefinedScalarFunctions.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/utils/userDefinedScalarFunctions.scala index d45a71ba40c..e1468bead66 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/utils/userDefinedScalarFunctions.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/utils/userDefinedScalarFunctions.scala @@ -20,7 +20,8 @@ package org.apache.flink.table.planner.expressions.utils import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.annotation.DataTypeHint import org.apache.flink.table.api.Types -import org.apache.flink.table.functions.{FunctionContext, ScalarFunction} +import org.apache.flink.table.functions.{AggregateFunction, FunctionContext, ScalarFunction, TableFunction} +import org.apache.flink.table.planner.utils.CountAccumulator import org.apache.flink.types.Row import org.apache.commons.lang3.StringUtils @@ -232,3 +233,53 @@ class SplitUDF(deterministic: Boolean) extends ScalarFunction { override def isDeterministic: Boolean = deterministic } + +@SerialVersionUID(1L) +class TestNonDeterministicUdf extends ScalarFunction { + val random = new Random() + + def eval(id: JLong): JLong = { +id + random.nextInt() + } + + def eval(id: Int): Int = { +id + random.nextInt() + } + + def eval(id: String): String = { +s"$id-${random.nextInt()}" + } + + override def isDeterministic: Boolean = false +} + +@SerialVersionUID(1L) +class TestNonDeterministicUdtf extends TableFunction[String] { + + val random = new Random() + + def eval(id: Int): Unit = { +collect(s"${id + random.nextInt()}") + } + + def eval(id: String): Unit = { +id.split(",").foreach(str => collect(s"$str#${random.nextInt()}")) + } + + override def isDeterministic: Boolean = false +} + +class TestNonDeterministicUdaf extends AggregateFunction[JLong, CountAccumulator] { + + val random = new Random() + + def accumulate(acc: CountAccumulator, in: JLong): Unit = { +acc.f0 += (in + random.nextInt()) + } + + override def getValue(acc: CountAccumulator): JLong = acc.f0 + + override def createAccumulator(): CountAccumulator = new CountAccumulator + + override def isDeterministic: Boolean = false +} diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/NonDeterministicDagTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/NonDeterministicDagTest.scala index c86bb1c07e4..dbd44247dd0 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/NonDeterministicDagTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/NonDeterministicDagTest.scala @@ -26,10 +26,10 @@ import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfi import org.apache.flink.table.api.config.OptimizerConfigOptions.NonDeterministicUpdateStrategy import org.apache.flink.table.api.internal.TableEnvironmentInternal import org.apache.flink.table.data.RowData -import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, TableFunction} -import org.apache.flink.table.planner.{JBoolean, JLong} +import org.apache.flink.table.planner.JBoolean +import org.apache.flink.table.planner.expressions.utils.{TestNonDeterministicUdaf, TestNonDeterministicUdf, TestNonDeterministicUdtf} import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedTableFunctions.StringSplit -import org.apache.flink.table.planner.utils.{CountAccumulator, StreamTableTestUtil, TableTestBase} +import org.apache.flink.table.planner.utils.{StreamTableTestUtil, TableTestBase} import org.apache.flink.table.runtime.typeutils.InternalTypeInfo import org.apache.flink.table.sinks.UpsertStreamTableSink import org.apache.flink.table.types.DataType @@ -42,8 +42,6 @@ import org.junit.runners.Parameterized import java.util -import scala.util.Random
[flink] 03/03: [FLINK-28569][table-planner] Fix SinkUpsertMaterializer that should be aware of the input upsertKey if it is not empty to prevent wrong results
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git commit 390612320fa9be297d9eed1f4b75f8ba2ec83c40 Author: lincoln.lil AuthorDate: Thu Sep 8 21:50:10 2022 +0800 [FLINK-28569][table-planner] Fix SinkUpsertMaterializer that should be aware of the input upsertKey if it is not empty to prevent wrong results This closes #20791 (cherry picked from commit bff1fc2782e04598275ab05177b89df38f891c94) --- .../plan/nodes/exec/batch/BatchExecSink.java | 3 +- .../plan/nodes/exec/common/CommonExecSink.java | 25 +++- .../plan/nodes/exec/stream/StreamExecSink.java | 12 +- .../table/planner/plan/utils/UpsertKeyUtil.java| 66 + .../nodes/physical/stream/StreamPhysicalSink.scala | 10 +- .../nodes/exec/stream/TableSinkJsonPlanTest.java | 36 + .../planner/plan/utils/UpsertKeyUtilTest.java | 52 .../utils/JavaUserDefinedScalarFunctions.java | 4 + .../testChangelogSource.out| 1 + .../testUpsertSource.out | 1 + .../testDeduplication.out | 1 + .../ExpandJsonPlanTest_jsonplan/testExpand.out | 1 + ...tDistinctAggCalls[isMiniBatchEnabled=false].out | 1 + ...stDistinctAggCalls[isMiniBatchEnabled=true].out | 1 + ...gCallsWithGroupBy[isMiniBatchEnabled=false].out | 1 + ...ggCallsWithGroupBy[isMiniBatchEnabled=true].out | 1 + ...erDefinedAggCalls[isMiniBatchEnabled=false].out | 1 + ...serDefinedAggCalls[isMiniBatchEnabled=true].out | 1 + .../testEventTimeTumbleWindow.out | 1 + .../testProcTimeTumbleWindow.out | 1 + .../testIncrementalAggregate.out | 1 + ...lAggregateWithSumCountDistinctAndRetraction.out | 1 + .../testInnerJoinWithEqualPk.out | 1 + ...WithNonDeterministicFuncSinkWithDifferentPk.out | 147 + .../runtime/stream/sql/TableSinkITCase.scala | 62 + .../operators/sink/SinkUpsertMaterializer.java | 125 +- .../operators/sink/SinkUpsertMaterializerTest.java | 102 +++--- 27 files changed, 601 insertions(+), 58 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSink.java index a6202871816..c9068656a51 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSink.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSink.java @@ -71,6 +71,7 @@ public class BatchExecSink extends CommonExecSink implements BatchExecNode Transformation inputTransform, DynamicTableSink tableSink, int rowtimeFieldIndex, -boolean upsertMaterialize) { +boolean upsertMaterialize, +int[] inputUpsertKey) { final ResolvedSchema schema = tableSinkSpec.getContextResolvedTable().getResolvedSchema(); final SinkRuntimeProvider runtimeProvider = tableSink.getSinkRuntimeProvider(new SinkRuntimeProviderContext(isBounded)); @@ -193,7 +195,8 @@ public abstract class CommonExecSink extends ExecNodeBase sinkParallelism, config, classLoader, -physicalRowType); +physicalRowType, +inputUpsertKey); } return (Transformation) @@ -402,16 +405,28 @@ public abstract class CommonExecSink extends ExecNodeBase int sinkParallelism, ExecNodeConfig config, ClassLoader classLoader, -RowType physicalRowType) { -GeneratedRecordEqualiser equaliser = +RowType physicalRowType, +int[] inputUpsertKey) { +final GeneratedRecordEqualiser rowEqualiser = new EqualiserCodeGenerator(physicalRowType, classLoader) .generateRecordEqualiser("SinkMaterializeEqualiser"); +final GeneratedRecordEqualiser upsertKeyEqualiser = +inputUpsertKey == null +? null +: new EqualiserCodeGenerator( +RowTypeUtils.projectRowType( +physicalRowType, inputUpsertKey), +classLoader) + .generateRecordEqualiser("SinkMaterializeUpsertKeyEqualiser"); + SinkUpsertMaterializer operator = new Si
[flink] 02/03: [FLINK-28569][table-planner] Move non-deterministic test functions to userDefinedScalarFunctions
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 7df115c016cab326627e4c12b2a4c449c1794e95 Author: lincoln.lil AuthorDate: Thu Sep 8 18:11:15 2022 +0800 [FLINK-28569][table-planner] Move non-deterministic test functions to userDefinedScalarFunctions This closes #20791 --- .../utils/userDefinedScalarFunctions.scala | 53 +- .../plan/stream/sql/NonDeterministicDagTest.scala | 64 ++ 2 files changed, 58 insertions(+), 59 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/utils/userDefinedScalarFunctions.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/utils/userDefinedScalarFunctions.scala index d45a71ba40c..e1468bead66 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/utils/userDefinedScalarFunctions.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/utils/userDefinedScalarFunctions.scala @@ -20,7 +20,8 @@ package org.apache.flink.table.planner.expressions.utils import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.annotation.DataTypeHint import org.apache.flink.table.api.Types -import org.apache.flink.table.functions.{FunctionContext, ScalarFunction} +import org.apache.flink.table.functions.{AggregateFunction, FunctionContext, ScalarFunction, TableFunction} +import org.apache.flink.table.planner.utils.CountAccumulator import org.apache.flink.types.Row import org.apache.commons.lang3.StringUtils @@ -232,3 +233,53 @@ class SplitUDF(deterministic: Boolean) extends ScalarFunction { override def isDeterministic: Boolean = deterministic } + +@SerialVersionUID(1L) +class TestNonDeterministicUdf extends ScalarFunction { + val random = new Random() + + def eval(id: JLong): JLong = { +id + random.nextInt() + } + + def eval(id: Int): Int = { +id + random.nextInt() + } + + def eval(id: String): String = { +s"$id-${random.nextInt()}" + } + + override def isDeterministic: Boolean = false +} + +@SerialVersionUID(1L) +class TestNonDeterministicUdtf extends TableFunction[String] { + + val random = new Random() + + def eval(id: Int): Unit = { +collect(s"${id + random.nextInt()}") + } + + def eval(id: String): Unit = { +id.split(",").foreach(str => collect(s"$str#${random.nextInt()}")) + } + + override def isDeterministic: Boolean = false +} + +class TestNonDeterministicUdaf extends AggregateFunction[JLong, CountAccumulator] { + + val random = new Random() + + def accumulate(acc: CountAccumulator, in: JLong): Unit = { +acc.f0 += (in + random.nextInt()) + } + + override def getValue(acc: CountAccumulator): JLong = acc.f0 + + override def createAccumulator(): CountAccumulator = new CountAccumulator + + override def isDeterministic: Boolean = false +} diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/NonDeterministicDagTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/NonDeterministicDagTest.scala index c86bb1c07e4..dbd44247dd0 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/NonDeterministicDagTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/NonDeterministicDagTest.scala @@ -26,10 +26,10 @@ import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfi import org.apache.flink.table.api.config.OptimizerConfigOptions.NonDeterministicUpdateStrategy import org.apache.flink.table.api.internal.TableEnvironmentInternal import org.apache.flink.table.data.RowData -import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, TableFunction} -import org.apache.flink.table.planner.{JBoolean, JLong} +import org.apache.flink.table.planner.JBoolean +import org.apache.flink.table.planner.expressions.utils.{TestNonDeterministicUdaf, TestNonDeterministicUdf, TestNonDeterministicUdtf} import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedTableFunctions.StringSplit -import org.apache.flink.table.planner.utils.{CountAccumulator, StreamTableTestUtil, TableTestBase} +import org.apache.flink.table.planner.utils.{StreamTableTestUtil, TableTestBase} import org.apache.flink.table.runtime.typeutils.InternalTypeInfo import org.apache.flink.table.sinks.UpsertStreamTableSink import org.apache.flink.table.types.DataType @@ -42,8 +42,6 @@ import org.junit.runners.Parameterized import java.util -import scala.util.Random - @RunWith(classOf[Parameterized]) class NonDeterministicDagTest(nonDeterministicUpdateStrategy:
[flink] branch master updated (68ec91bdc3b -> bff1fc2782e)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 68ec91bdc3b [hotfix][tests] Update job name new dbcd2d7b86f [FLINK-28569][table-planner] Add projectRowType to RowTypeUtils and deprecate AggCodeGenHelper#projectRowType new 7df115c016c [FLINK-28569][table-planner] Move non-deterministic test functions to userDefinedScalarFunctions new bff1fc2782e [FLINK-28569][table-planner] Fix SinkUpsertMaterializer that should be aware of the input upsertKey if it is not empty to prevent wrong results 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: .../plan/nodes/exec/batch/BatchExecSink.java | 3 +- .../plan/nodes/exec/common/CommonExecSink.java | 25 +++- .../plan/nodes/exec/stream/StreamExecSink.java | 12 +- .../table/planner/plan/utils/UpsertKeyUtil.java| 66 + .../table/planner/typeutils/RowTypeUtils.java | 35 + .../codegen/agg/batch/AggCodeGenHelper.scala | 4 - .../codegen/agg/batch/HashAggCodeGenerator.scala | 3 +- .../codegen/agg/batch/SortAggCodeGenerator.scala | 3 +- .../codegen/agg/batch/WindowCodeGenerator.scala| 3 +- .../nodes/physical/stream/StreamPhysicalSink.scala | 10 +- .../nodes/exec/stream/TableSinkJsonPlanTest.java | 36 + .../planner/plan/utils/UpsertKeyUtilTest.java | 52 .../utils/JavaUserDefinedScalarFunctions.java | 4 + .../table/planner/typeutils/RowTypeUtilsTest.java | 46 +++ .../testChangelogSource.out| 1 + .../testUpsertSource.out | 1 + .../testDeduplication.out | 1 + .../ExpandJsonPlanTest_jsonplan/testExpand.out | 1 + ...tDistinctAggCalls[isMiniBatchEnabled=false].out | 1 + ...stDistinctAggCalls[isMiniBatchEnabled=true].out | 1 + ...gCallsWithGroupBy[isMiniBatchEnabled=false].out | 1 + ...ggCallsWithGroupBy[isMiniBatchEnabled=true].out | 1 + ...erDefinedAggCalls[isMiniBatchEnabled=false].out | 1 + ...serDefinedAggCalls[isMiniBatchEnabled=true].out | 1 + .../testEventTimeTumbleWindow.out | 1 + .../testProcTimeTumbleWindow.out | 1 + .../testIncrementalAggregate.out | 1 + ...lAggregateWithSumCountDistinctAndRetraction.out | 1 + .../testInnerJoinWithEqualPk.out | 1 + ...WithNonDeterministicFuncSinkWithDifferentPk.out | 147 + .../utils/userDefinedScalarFunctions.scala | 53 +++- .../plan/stream/sql/NonDeterministicDagTest.scala | 64 + .../runtime/stream/sql/TableSinkITCase.scala | 62 + .../operators/sink/SinkUpsertMaterializer.java | 125 +- .../operators/sink/SinkUpsertMaterializerTest.java | 102 +++--- 35 files changed, 746 insertions(+), 124 deletions(-) create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/UpsertKeyUtil.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/UpsertKeyUtilTest.java create mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testCdcWithNonDeterministicFuncSinkWithDifferentPk.out
[flink] 01/03: [FLINK-28569][table-planner] Add projectRowType to RowTypeUtils and deprecate AggCodeGenHelper#projectRowType
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit dbcd2d7b86fcb7fa7a26e181f1719ea4c6dad828 Author: lincoln.lil AuthorDate: Thu Sep 8 18:08:58 2022 +0800 [FLINK-28569][table-planner] Add projectRowType to RowTypeUtils and deprecate AggCodeGenHelper#projectRowType This closes #20791 --- .../table/planner/typeutils/RowTypeUtils.java | 35 .../codegen/agg/batch/AggCodeGenHelper.scala | 4 -- .../codegen/agg/batch/HashAggCodeGenerator.scala | 3 +- .../codegen/agg/batch/SortAggCodeGenerator.scala | 3 +- .../codegen/agg/batch/WindowCodeGenerator.scala| 3 +- .../table/planner/typeutils/RowTypeUtilsTest.java | 46 ++ 6 files changed, 87 insertions(+), 7 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/typeutils/RowTypeUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/typeutils/RowTypeUtils.java index ffb9a68a131..4d9879d7b99 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/typeutils/RowTypeUtils.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/typeutils/RowTypeUtils.java @@ -18,7 +18,13 @@ package org.apache.flink.table.planner.typeutils; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; + +import javax.annotation.Nonnull; + import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -46,4 +52,33 @@ public class RowTypeUtils { } return result; } + +/** + * Returns projected {@link RowType} by given projection indexes over original {@link RowType}. + * Will raise an error when projection index beyond the field count of original rowType. + * + * @param rowType source row type + * @param projection indexes array + * @return projected {@link RowType} + */ +public static RowType projectRowType(@Nonnull RowType rowType, @Nonnull int[] projection) +throws IllegalArgumentException { +final int fieldCnt = rowType.getFieldCount(); +return RowType.of( +Arrays.stream(projection) +.mapToObj( +index -> { +if (index >= fieldCnt) { +throw new IllegalArgumentException( +String.format( +"Invalid projection index: %d of source rowType size: %d", +index, fieldCnt)); +} +return rowType.getTypeAt(index); +}) +.toArray(LogicalType[]::new), +Arrays.stream(projection) +.mapToObj(index -> rowType.getFieldNames().get(index)) +.toArray(String[]::new)); +} } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/AggCodeGenHelper.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/AggCodeGenHelper.scala index bb1135fc236..c401e50ea24 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/AggCodeGenHelper.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/AggCodeGenHelper.scala @@ -93,10 +93,6 @@ object AggCodeGenHelper { .asInstanceOf[Map[AggregateFunction[_, _], String]] } - def projectRowType(rowType: RowType, mapping: Array[Int]): RowType = { -RowType.of(mapping.map(rowType.getTypeAt), mapping.map(rowType.getFieldNames.get(_))) - } - /** Add agg handler to class member and open it. */ private[flink] def addAggsHandler( aggsHandler: GeneratedAggsHandleFunction, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/HashAggCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/HashAggCodeGenerator.scala index 550a93df3fb..c768ffd5f70 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/HashAggCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/HashAggCodeGenerator.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.functions.AggregateFunction import org.apache.flink.table.planner.codegen.{CodeGeneratorContext, CodeGenUtils, ProjectionCod
[flink] 03/03: [FLINK-28569][table-planner] Fix SinkUpsertMaterializer that should be aware of the input upsertKey if it is not empty to prevent wrong results
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit bff1fc2782e04598275ab05177b89df38f891c94 Author: lincoln.lil AuthorDate: Thu Sep 8 21:50:10 2022 +0800 [FLINK-28569][table-planner] Fix SinkUpsertMaterializer that should be aware of the input upsertKey if it is not empty to prevent wrong results This closes #20791 --- .../plan/nodes/exec/batch/BatchExecSink.java | 3 +- .../plan/nodes/exec/common/CommonExecSink.java | 25 +++- .../plan/nodes/exec/stream/StreamExecSink.java | 12 +- .../table/planner/plan/utils/UpsertKeyUtil.java| 66 + .../nodes/physical/stream/StreamPhysicalSink.scala | 10 +- .../nodes/exec/stream/TableSinkJsonPlanTest.java | 36 + .../planner/plan/utils/UpsertKeyUtilTest.java | 52 .../utils/JavaUserDefinedScalarFunctions.java | 4 + .../testChangelogSource.out| 1 + .../testUpsertSource.out | 1 + .../testDeduplication.out | 1 + .../ExpandJsonPlanTest_jsonplan/testExpand.out | 1 + ...tDistinctAggCalls[isMiniBatchEnabled=false].out | 1 + ...stDistinctAggCalls[isMiniBatchEnabled=true].out | 1 + ...gCallsWithGroupBy[isMiniBatchEnabled=false].out | 1 + ...ggCallsWithGroupBy[isMiniBatchEnabled=true].out | 1 + ...erDefinedAggCalls[isMiniBatchEnabled=false].out | 1 + ...serDefinedAggCalls[isMiniBatchEnabled=true].out | 1 + .../testEventTimeTumbleWindow.out | 1 + .../testProcTimeTumbleWindow.out | 1 + .../testIncrementalAggregate.out | 1 + ...lAggregateWithSumCountDistinctAndRetraction.out | 1 + .../testInnerJoinWithEqualPk.out | 1 + ...WithNonDeterministicFuncSinkWithDifferentPk.out | 147 + .../runtime/stream/sql/TableSinkITCase.scala | 62 + .../operators/sink/SinkUpsertMaterializer.java | 125 +- .../operators/sink/SinkUpsertMaterializerTest.java | 102 +++--- 27 files changed, 601 insertions(+), 58 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSink.java index a6202871816..c9068656a51 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSink.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSink.java @@ -71,6 +71,7 @@ public class BatchExecSink extends CommonExecSink implements BatchExecNode Transformation inputTransform, DynamicTableSink tableSink, int rowtimeFieldIndex, -boolean upsertMaterialize) { +boolean upsertMaterialize, +int[] inputUpsertKey) { final ResolvedSchema schema = tableSinkSpec.getContextResolvedTable().getResolvedSchema(); final SinkRuntimeProvider runtimeProvider = tableSink.getSinkRuntimeProvider(new SinkRuntimeProviderContext(isBounded)); @@ -193,7 +195,8 @@ public abstract class CommonExecSink extends ExecNodeBase sinkParallelism, config, classLoader, -physicalRowType); +physicalRowType, +inputUpsertKey); } return (Transformation) @@ -402,16 +405,28 @@ public abstract class CommonExecSink extends ExecNodeBase int sinkParallelism, ExecNodeConfig config, ClassLoader classLoader, -RowType physicalRowType) { -GeneratedRecordEqualiser equaliser = +RowType physicalRowType, +int[] inputUpsertKey) { +final GeneratedRecordEqualiser rowEqualiser = new EqualiserCodeGenerator(physicalRowType, classLoader) .generateRecordEqualiser("SinkMaterializeEqualiser"); +final GeneratedRecordEqualiser upsertKeyEqualiser = +inputUpsertKey == null +? null +: new EqualiserCodeGenerator( +RowTypeUtils.projectRowType( +physicalRowType, inputUpsertKey), +classLoader) + .generateRecordEqualiser("SinkMaterializeUpsertKeyEqualiser"); + SinkUpsertMaterializer operator = new SinkUpsertMaterializer( StateConfigUtil.createTtlConfig(
[flink] branch release-1.16 updated: [FLINK-29113][table-planner] Throw exception when finding at least one invalid table name in join hints
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.16 by this push: new 0e02c082037 [FLINK-29113][table-planner] Throw exception when finding at least one invalid table name in join hints 0e02c082037 is described below commit 0e02c082037979766b61da90e37f5fc555d71770 Author: xuyang AuthorDate: Mon Aug 29 18:21:39 2022 +0800 [FLINK-29113][table-planner] Throw exception when finding at least one invalid table name in join hints This closes #20705 (cherry picked from commit d518086f475ec92a18592ec3c423bf6398e776cf) --- .../planner/plan/optimize/JoinHintResolver.java| 113 ++--- .../planner/plan/hints/batch/JoinHintTestBase.java | 45 +++- .../plan/stream/sql/join/LookupJoinTest.scala | 3 +- 3 files changed, 144 insertions(+), 17 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolver.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolver.java index ba11c29c1c9..36af173b1ae 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolver.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolver.java @@ -34,10 +34,14 @@ import org.apache.calcite.rel.logical.LogicalJoin; import org.apache.commons.lang3.StringUtils; import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import static java.util.Collections.singletonList; @@ -52,6 +56,9 @@ public class JoinHintResolver extends RelShuttleImpl { private final Set allHints = new HashSet<>(); private final Set validHints = new HashSet<>(); +// hintName -> hintOptions -> whether this option has been checked +private final Map> allOptionsInJoinHints = new HashMap<>(); + /** * Resolves and validates join hints in the given {@link RelNode} list, an {@link * ValidationException} will be raised for invalid hints. @@ -64,7 +71,7 @@ public class JoinHintResolver extends RelShuttleImpl { * right side of this join, that means this join hint is invalid and a {@link * ValidationException} will be thrown. */ -public List resolve(List roots) { +final List resolve(List roots) { List resolvedRoots = roots.stream().map(node -> node.accept(this)).collect(Collectors.toList()); validateHints(); @@ -96,13 +103,22 @@ public class JoinHintResolver extends RelShuttleImpl { Configuration conf = Configuration.fromMap(hint.kvOptions); // hint option checker has done the validation String lookupTable = conf.get(LOOKUP_TABLE); + +// add options about this hint for finally checking +initOptionInfoAboutJoinHintsForCheck( +hint.hintName, Collections.singletonList(lookupTable)); + assert null != lookupTable; if (rightName.isPresent() && matchIdentifier(lookupTable, rightName.get())) { validHints.add(trimInheritPath(hint)); +updateInfoForOptionCheck(hint.hintName, rightName); newHints.add(hint); } } else if (JoinStrategy.isJoinStrategy(hint.hintName)) { allHints.add(trimInheritPath(hint)); +// add options about this hint for finally checking +initOptionInfoAboutJoinHintsForCheck(hint.hintName, hint.listOptions); + // the declared table name or query block name is replaced by // JoinStrategy#LEFT_INPUT or JoinStrategy#RIGHT_INPUT List newOptions = @@ -143,6 +159,11 @@ public class JoinHintResolver extends RelShuttleImpl { Optional rightName, List listOptions, String hintName) { + +// update info about 'allOptionsInJoinHints' for checking finally +updateInfoForOptionCheck(hintName, leftName); +updateInfoForOptionCheck(hintName, rightName); + return listOptions.stream() .map( option -> { @@ -172,21 +193,57 @@ public class JoinHintResolver extends RelShuttleImpl { private void validateHints() { Set invalidHints = new HashSet<>(allHints); invalidHints.removeAll(validHints); +String errorPattern; + +// firstly, check the unknown ta
[flink] branch master updated (030baed8d6e -> d518086f475)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 030baed8d6e [FLINK-29262][docs] Document API compatibility guarantees add d518086f475 [FLINK-29113][table-planner] Throw exception when finding at least one invalid table name in join hints No new revisions were added by this update. Summary of changes: .../planner/plan/optimize/JoinHintResolver.java| 113 ++--- .../planner/plan/hints/batch/JoinHintTestBase.java | 45 +++- .../plan/stream/sql/join/LookupJoinTest.scala | 3 +- 3 files changed, 144 insertions(+), 17 deletions(-)
[flink] branch release-1.16 updated: [FLINK-29120][table-planner] Avoid join hint propagating into view
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.16 by this push: new 357221b9b35 [FLINK-29120][table-planner] Avoid join hint propagating into view 357221b9b35 is described below commit 357221b9b3543447b2b439a413639c0ed201ab35 Author: xuyang AuthorDate: Mon Aug 29 15:12:36 2022 +0800 [FLINK-29120][table-planner] Avoid join hint propagating into view This closes #20697 (cherry picked from commit 6722c89d0df35643dde38c1b8f096aa785579884) --- .../table/planner/calcite/FlinkPlannerImpl.scala | 29 ++- .../planner/plan/hints/batch/JoinHintTestBase.java | 91 ++- .../plan/hints/batch/BroadcastJoinHintTest.xml | 289 - .../plan/hints/batch/NestLoopJoinHintTest.xml | 289 - .../plan/hints/batch/ShuffleHashJoinHintTest.xml | 289 - .../plan/hints/batch/ShuffleMergeJoinHintTest.xml | 289 - .../optimize/ClearQueryBlockAliasResolverTest.xml | 155 ++- .../planner/plan/optimize/JoinHintResolverTest.xml | 159 +++- 8 files changed, 1544 insertions(+), 46 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala index f9488750b53..07969d9e68d 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala @@ -193,17 +193,22 @@ class FlinkPlannerImpl( private def rel(validatedSqlNode: SqlNode, sqlValidator: FlinkCalciteSqlValidator) = { try { assert(validatedSqlNode != null) - val sqlToRelConverter: SqlToRelConverter = createSqlToRelConverter(sqlValidator) - // check whether this SqlNode tree contains join hints val checkContainJoinHintShuttle = new CheckContainJoinHintShuttle validatedSqlNode.accept(checkContainJoinHintShuttle) - checkContainJoinHintShuttle.containsJoinHint - - // TODO currently, it is a relatively hacked way to tell converter - // that this SqlNode tree contains join hints - if (checkContainJoinHintShuttle.containsJoinHint) { -sqlToRelConverter.containsJoinHint() + val sqlToRelConverter: SqlToRelConverter = if (checkContainJoinHintShuttle.containsJoinHint) { +val converter = createSqlToRelConverter( + sqlValidator, + // disable project merge during sql to rel phase to prevent + // incorrect propagation of join hints into child query block + sqlToRelConverterConfig.addRelBuilderConfigTransform(c => c.withBloat(-1)) +) +// TODO currently, it is a relatively hacked way to tell converter +// that this SqlNode tree contains join hints +converter.containsJoinHint() +converter + } else { +createSqlToRelConverter(sqlValidator, sqlToRelConverterConfig) } sqlToRelConverter.convertQuery(validatedSqlNode, false, true) @@ -301,7 +306,7 @@ class FlinkPlannerImpl( @Nullable outputType: RelDataType) = { try { val validatedSqlNode = validateExpression(sqlNode, sqlValidator, inputRowType, outputType) - val sqlToRelConverter = createSqlToRelConverter(sqlValidator) + val sqlToRelConverter = createSqlToRelConverter(sqlValidator, sqlToRelConverterConfig) val nameToNodeMap = inputRowType.getFieldList.asScala .map(field => (field.getName, RexInputRef.of(field.getIndex, inputRowType))) .toMap[String, RexNode] @@ -312,14 +317,16 @@ class FlinkPlannerImpl( } } - private def createSqlToRelConverter(sqlValidator: SqlValidator): SqlToRelConverter = { + private def createSqlToRelConverter( + sqlValidator: SqlValidator, + config: SqlToRelConverter.Config): SqlToRelConverter = { new SqlToRelConverter( createToRelContext(), sqlValidator, sqlValidator.getCatalogReader.unwrap(classOf[CalciteCatalogReader]), cluster, convertletTable, - sqlToRelConverterConfig) + config) } /** Creates a new instance of [[RelOptTable.ToRelContext]] for [[RelOptTable]]. */ diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/JoinHintTestBase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/JoinHintTestBase.java index 4af17d5d4fa..c20fe3eb2ff 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/JoinHintTestBase.java +++ b/flink-table/flink-table-planner/src/test/java/org/
[flink] branch master updated: [FLINK-29120][table-planner] Avoid join hint propagating into view
This is an automated email from the ASF dual-hosted git repository. godfrey 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 6722c89d0df [FLINK-29120][table-planner] Avoid join hint propagating into view 6722c89d0df is described below commit 6722c89d0df35643dde38c1b8f096aa785579884 Author: xuyang AuthorDate: Mon Aug 29 15:12:36 2022 +0800 [FLINK-29120][table-planner] Avoid join hint propagating into view This closes #20697 --- .../table/planner/calcite/FlinkPlannerImpl.scala | 29 ++- .../planner/plan/hints/batch/JoinHintTestBase.java | 91 ++- .../plan/hints/batch/BroadcastJoinHintTest.xml | 289 - .../plan/hints/batch/NestLoopJoinHintTest.xml | 289 - .../plan/hints/batch/ShuffleHashJoinHintTest.xml | 289 - .../plan/hints/batch/ShuffleMergeJoinHintTest.xml | 289 - .../optimize/ClearQueryBlockAliasResolverTest.xml | 155 ++- .../planner/plan/optimize/JoinHintResolverTest.xml | 159 +++- 8 files changed, 1544 insertions(+), 46 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala index f9488750b53..07969d9e68d 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala @@ -193,17 +193,22 @@ class FlinkPlannerImpl( private def rel(validatedSqlNode: SqlNode, sqlValidator: FlinkCalciteSqlValidator) = { try { assert(validatedSqlNode != null) - val sqlToRelConverter: SqlToRelConverter = createSqlToRelConverter(sqlValidator) - // check whether this SqlNode tree contains join hints val checkContainJoinHintShuttle = new CheckContainJoinHintShuttle validatedSqlNode.accept(checkContainJoinHintShuttle) - checkContainJoinHintShuttle.containsJoinHint - - // TODO currently, it is a relatively hacked way to tell converter - // that this SqlNode tree contains join hints - if (checkContainJoinHintShuttle.containsJoinHint) { -sqlToRelConverter.containsJoinHint() + val sqlToRelConverter: SqlToRelConverter = if (checkContainJoinHintShuttle.containsJoinHint) { +val converter = createSqlToRelConverter( + sqlValidator, + // disable project merge during sql to rel phase to prevent + // incorrect propagation of join hints into child query block + sqlToRelConverterConfig.addRelBuilderConfigTransform(c => c.withBloat(-1)) +) +// TODO currently, it is a relatively hacked way to tell converter +// that this SqlNode tree contains join hints +converter.containsJoinHint() +converter + } else { +createSqlToRelConverter(sqlValidator, sqlToRelConverterConfig) } sqlToRelConverter.convertQuery(validatedSqlNode, false, true) @@ -301,7 +306,7 @@ class FlinkPlannerImpl( @Nullable outputType: RelDataType) = { try { val validatedSqlNode = validateExpression(sqlNode, sqlValidator, inputRowType, outputType) - val sqlToRelConverter = createSqlToRelConverter(sqlValidator) + val sqlToRelConverter = createSqlToRelConverter(sqlValidator, sqlToRelConverterConfig) val nameToNodeMap = inputRowType.getFieldList.asScala .map(field => (field.getName, RexInputRef.of(field.getIndex, inputRowType))) .toMap[String, RexNode] @@ -312,14 +317,16 @@ class FlinkPlannerImpl( } } - private def createSqlToRelConverter(sqlValidator: SqlValidator): SqlToRelConverter = { + private def createSqlToRelConverter( + sqlValidator: SqlValidator, + config: SqlToRelConverter.Config): SqlToRelConverter = { new SqlToRelConverter( createToRelContext(), sqlValidator, sqlValidator.getCatalogReader.unwrap(classOf[CalciteCatalogReader]), cluster, convertletTable, - sqlToRelConverterConfig) + config) } /** Creates a new instance of [[RelOptTable.ToRelContext]] for [[RelOptTable]]. */ diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/JoinHintTestBase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/JoinHintTestBase.java index 4af17d5d4fa..c20fe3eb2ff 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/JoinHintTestBase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/JoinHintTestBase.java @@ -95,6 +95,9 @@ public ab
[flink] branch release-1.16 updated: [FLINK-28787][table-planner] Rename getUniqueKeys to getUpsertKeys in CommonPhysicalJoin
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.16 by this push: new 77574b7e1ee [FLINK-28787][table-planner] Rename getUniqueKeys to getUpsertKeys in CommonPhysicalJoin 77574b7e1ee is described below commit 77574b7e1ee5f514c32304c071732a11050baede Author: lincoln lee AuthorDate: Tue Sep 6 16:58:08 2022 +0800 [FLINK-28787][table-planner] Rename getUniqueKeys to getUpsertKeys in CommonPhysicalJoin This closes #20763 (cherry picked from commit 347316ea6394b24c4471aa8616f2632e126f733d) --- .../plan/nodes/exec/stream/StreamExecJoin.java | 35 -- .../StreamNonDeterministicUpdatePlanVisitor.java | 4 +-- .../nodes/physical/common/CommonPhysicalJoin.scala | 3 +- .../nodes/physical/stream/StreamPhysicalJoin.scala | 10 +++ .../JoinJsonPlanTest_jsonplan/testInnerJoin.out| 2 -- .../testInnerJoinWithEqualPk.out | 4 +-- .../testInnerJoinWithPk.out| 4 +-- .../testLeftJoinNonEqui.out| 2 -- 8 files changed, 31 insertions(+), 33 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java index b8a45b8d6f1..dbf399d8ae5 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java @@ -48,6 +48,7 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonInclude; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; @@ -73,23 +74,25 @@ public class StreamExecJoin extends ExecNodeBase public static final String JOIN_TRANSFORMATION = "join"; public static final String FIELD_NAME_JOIN_SPEC = "joinSpec"; -public static final String FIELD_NAME_LEFT_UNIQUE_KEYS = "leftUniqueKeys"; -public static final String FIELD_NAME_RIGHT_UNIQUE_KEYS = "rightUniqueKeys"; +public static final String FIELD_NAME_LEFT_UPSERT_KEYS = "leftUpsertKeys"; +public static final String FIELD_NAME_RIGHT_UPSERT_KEYS = "rightUpsertKeys"; @JsonProperty(FIELD_NAME_JOIN_SPEC) private final JoinSpec joinSpec; -@JsonProperty(FIELD_NAME_LEFT_UNIQUE_KEYS) -private final List leftUniqueKeys; +@JsonProperty(FIELD_NAME_LEFT_UPSERT_KEYS) +@JsonInclude(JsonInclude.Include.NON_DEFAULT) +private final List leftUpsertKeys; -@JsonProperty(FIELD_NAME_RIGHT_UNIQUE_KEYS) -private final List rightUniqueKeys; +@JsonProperty(FIELD_NAME_RIGHT_UPSERT_KEYS) +@JsonInclude(JsonInclude.Include.NON_DEFAULT) +private final List rightUpsertKeys; public StreamExecJoin( ReadableConfig tableConfig, JoinSpec joinSpec, -List leftUniqueKeys, -List rightUniqueKeys, +List leftUpsertKeys, +List rightUpsertKeys, InputProperty leftInputProperty, InputProperty rightInputProperty, RowType outputType, @@ -99,8 +102,8 @@ public class StreamExecJoin extends ExecNodeBase ExecNodeContext.newContext(StreamExecJoin.class), ExecNodeContext.newPersistedConfig(StreamExecJoin.class, tableConfig), joinSpec, -leftUniqueKeys, -rightUniqueKeys, +leftUpsertKeys, +rightUpsertKeys, Lists.newArrayList(leftInputProperty, rightInputProperty), outputType, description); @@ -112,16 +115,16 @@ public class StreamExecJoin extends ExecNodeBase @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_JOIN_SPEC) JoinSpec joinSpec, -@JsonProperty(FIELD_NAME_LEFT_UNIQUE_KEYS) List leftUniqueKeys, -@JsonProperty(FIELD_NAME_RIGHT_UNIQUE_KEYS) List rightUniqueKeys, +@JsonProperty(FIELD_NAME_LEFT_UPSERT_KEYS) List leftUpsertKeys, +@JsonProperty(FIELD_NAME_RIGHT_UPSERT_KEYS) List rightUpsertKeys, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties,
[flink] branch master updated: [FLINK-28787][table-planner] Rename getUniqueKeys to getUpsertKeys in CommonPhysicalJoin
This is an automated email from the ASF dual-hosted git repository. godfrey 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 347316ea639 [FLINK-28787][table-planner] Rename getUniqueKeys to getUpsertKeys in CommonPhysicalJoin 347316ea639 is described below commit 347316ea6394b24c4471aa8616f2632e126f733d Author: lincoln lee AuthorDate: Tue Sep 6 16:58:08 2022 +0800 [FLINK-28787][table-planner] Rename getUniqueKeys to getUpsertKeys in CommonPhysicalJoin This closes #20763 --- .../plan/nodes/exec/stream/StreamExecJoin.java | 35 -- .../StreamNonDeterministicUpdatePlanVisitor.java | 4 +-- .../nodes/physical/common/CommonPhysicalJoin.scala | 3 +- .../nodes/physical/stream/StreamPhysicalJoin.scala | 10 +++ .../JoinJsonPlanTest_jsonplan/testInnerJoin.out| 2 -- .../testInnerJoinWithEqualPk.out | 4 +-- .../testInnerJoinWithPk.out| 4 +-- .../testLeftJoinNonEqui.out| 2 -- 8 files changed, 31 insertions(+), 33 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java index b8a45b8d6f1..dbf399d8ae5 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java @@ -48,6 +48,7 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonInclude; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; @@ -73,23 +74,25 @@ public class StreamExecJoin extends ExecNodeBase public static final String JOIN_TRANSFORMATION = "join"; public static final String FIELD_NAME_JOIN_SPEC = "joinSpec"; -public static final String FIELD_NAME_LEFT_UNIQUE_KEYS = "leftUniqueKeys"; -public static final String FIELD_NAME_RIGHT_UNIQUE_KEYS = "rightUniqueKeys"; +public static final String FIELD_NAME_LEFT_UPSERT_KEYS = "leftUpsertKeys"; +public static final String FIELD_NAME_RIGHT_UPSERT_KEYS = "rightUpsertKeys"; @JsonProperty(FIELD_NAME_JOIN_SPEC) private final JoinSpec joinSpec; -@JsonProperty(FIELD_NAME_LEFT_UNIQUE_KEYS) -private final List leftUniqueKeys; +@JsonProperty(FIELD_NAME_LEFT_UPSERT_KEYS) +@JsonInclude(JsonInclude.Include.NON_DEFAULT) +private final List leftUpsertKeys; -@JsonProperty(FIELD_NAME_RIGHT_UNIQUE_KEYS) -private final List rightUniqueKeys; +@JsonProperty(FIELD_NAME_RIGHT_UPSERT_KEYS) +@JsonInclude(JsonInclude.Include.NON_DEFAULT) +private final List rightUpsertKeys; public StreamExecJoin( ReadableConfig tableConfig, JoinSpec joinSpec, -List leftUniqueKeys, -List rightUniqueKeys, +List leftUpsertKeys, +List rightUpsertKeys, InputProperty leftInputProperty, InputProperty rightInputProperty, RowType outputType, @@ -99,8 +102,8 @@ public class StreamExecJoin extends ExecNodeBase ExecNodeContext.newContext(StreamExecJoin.class), ExecNodeContext.newPersistedConfig(StreamExecJoin.class, tableConfig), joinSpec, -leftUniqueKeys, -rightUniqueKeys, +leftUpsertKeys, +rightUpsertKeys, Lists.newArrayList(leftInputProperty, rightInputProperty), outputType, description); @@ -112,16 +115,16 @@ public class StreamExecJoin extends ExecNodeBase @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_JOIN_SPEC) JoinSpec joinSpec, -@JsonProperty(FIELD_NAME_LEFT_UNIQUE_KEYS) List leftUniqueKeys, -@JsonProperty(FIELD_NAME_RIGHT_UNIQUE_KEYS) List rightUniqueKeys, +@JsonProperty(FIELD_NAME_LEFT_UPSERT_KEYS) List leftUpsertKeys, +@JsonProperty(FIELD_NAME_RIGHT_UPSERT_KEYS) List rightUpsertKeys, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_D
[flink] branch master updated: [hotfix][tests][table-planner] Add two more cases to verify the conflict of multiple LOOKUP hints
This is an automated email from the ASF dual-hosted git repository. godfrey 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 264eff6084d [hotfix][tests][table-planner] Add two more cases to verify the conflict of multiple LOOKUP hints 264eff6084d is described below commit 264eff6084ddf8f03105345ab9decd85ef78e475 Author: lincoln lee AuthorDate: Fri Sep 2 22:58:31 2022 +0800 [hotfix][tests][table-planner] Add two more cases to verify the conflict of multiple LOOKUP hints This closes #20743 --- .../plan/stream/sql/join/LookupJoinTest.xml| 170 ++--- .../plan/stream/sql/join/LookupJoinTest.scala | 30 2 files changed, 178 insertions(+), 22 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/LookupJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/LookupJoinTest.xml index b171bda90bf..c0626e94636 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/LookupJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/LookupJoinTest.xml @@ -1878,28 +1878,6 @@ Sink(table=[default_catalog.default_database.Sink1], fields=[a, name, age]) }]]> - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/LookupJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/LookupJoinTest.scala index 469b74fa287..943dfecdaf1 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/LookupJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/LookupJoinTest.scala @@ -766,6 +766,36 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri util.verifyExecPlan(sql) } + @Test + def testMultipleJoinHintsWithSameTableName(): Unit = { +// only the first hint will take effect +val sql = + """ +|SELECT /*+ LOOKUP('table'='AsyncLookupTable', 'output-mode'='allow_unordered'), +| LOOKUP('table'='AsyncLookupTable', 'output-mode'='ordered') */ * +|FROM MyTable AS T +|JOIN AsyncLookupTable FOR SYSTEM_TIME AS OF T.proctime AS D +| ON T.a = D.id + """.stripMargin +util.verifyExecPlan(sql) + } + + @Test + def testMultipleJoinHintsWithDifferentTableName(): Unit = { +// both hints on corresponding tables will take effect +val sql = + """ +|SELECT /*+ LOOKUP('table'='AsyncLookupTable', 'output-mode'='allow_unordered'), +| LOOKUP('table'='LookupTable', 'retry-predicate'='lookup_miss', 'retry-strategy'='fixed_delay', 'fixed-delay'='10s', 'max-attempts'='3') */ * +|FROM MyTable AS T +|JOIN AsyncLookupTable FOR SYSTEM_TIME AS OF T.proctime AS D +| ON T.a = D.id +|JOIN LookupTable FOR SYSTEM_TIME AS OF T.proctime AS D1 +| ON T.a = D1.id + """.stripMargin +util.verifyExecPlan(sql) + } + @Test def testJoinSyncTableWithAsyncHint(): Unit = { val sql =
[flink] branch master updated: [FLINK-29091][table-planner] Fix the determinism declaration of the rand function to be consistent with the current behavior
This is an automated email from the ASF dual-hosted git repository. godfrey 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 4b15bc900eb [FLINK-29091][table-planner] Fix the determinism declaration of the rand function to be consistent with the current behavior 4b15bc900eb is described below commit 4b15bc900eb60b1830bc406975ce974ad6050f98 Author: lincoln.lil AuthorDate: Wed Aug 24 18:00:09 2022 +0800 [FLINK-29091][table-planner] Fix the determinism declaration of the rand function to be consistent with the current behavior This closes #20674 --- docs/content.zh/docs/dev/table/functions/udfs.md | 53 ++ docs/content/docs/dev/table/functions/udfs.md | 48 +++- .../functions/sql/FlinkSqlOperatorTable.java | 44 +- .../rules/logical/ExpressionReductionRulesTest.xml | 17 +++ .../expressions/NonDeterministicTests.scala| 9 .../logical/ExpressionReductionRulesTest.scala | 6 +++ 6 files changed, 173 insertions(+), 4 deletions(-) diff --git a/docs/content.zh/docs/dev/table/functions/udfs.md b/docs/content.zh/docs/dev/table/functions/udfs.md index 6a3da037d07..ab619d06868 100644 --- a/docs/content.zh/docs/dev/table/functions/udfs.md +++ b/docs/content.zh/docs/dev/table/functions/udfs.md @@ -585,6 +585,59 @@ 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" >}}. +### 确定性 + +每个用户自定义函数类都可以通过重写 `isDeterministic()` 方法来声明它是否产生确定性的结果。如果该函数不是纯粹函数式的(如`random()`, `date()`, 或`now()`),该方法必须返回 `false`。默认情况下,`isDeterministic()` 返回 `true`。 + +此外,重写 `isDeterministic()` 方法也可能影响运行时行为。运行时实现可能会在两个不同的阶段被调用: + +1. **在生成执行计划期间**:如果一个函数是通过常量表达式调用的或者常量表达式可以从给定的语句中推导出来,那么一个函数就会被预计算以减少常量表达式,并且可能不再在集群上执行。 +除非 `isDeterministic()` 被重写为 `false` 用来在这种情况下禁用常量表达式简化。比如说,以下对 `ABS` 的调用在生成执行计划期间被执行:`SELECT ABS(-1) FROM t` 和 `SELECT ABS(field) FROM t WHERE field = -1`,而 `SELECT ABS(field) FROM t` 则不执行。 + +2. **在运行时(即在集群执行)**:如果一个函数被调用时带有非常量表达式或 `isDeterministic()` 返回 `false`。 + + 内置函数的确定性 +系统(内置)函数的确定性是不可改变的。存在两种不具有确定性的函数:动态函数和非确定性函数,根据 Apache Calcite `SqlOperator` 的定义: +```java + /** + * Returns whether a call to this operator is guaranteed to always return + * the same result given the same operands; true is assumed by default. + */ + public boolean isDeterministic() { +return true; + } + + /** + * Returns whether it is unsafe to cache query plans referencing this + * operator; false is assumed by default. + */ + public boolean isDynamicFunction() { +return false; + } +``` + +`isDeterministic` 表示函数的确定性,声明返回 `false` 时将在运行时对每个记录进行计算。 +`isDynamicFunction` 声明返回 `true` 时意味着该函数只能在查询开始时被计算,对于批处理模式,它只在生成执行计划期间被执行, +而对于流模式,它等效于一个非确定性的函数,这是因为查询在逻辑上是连续执行的(流模式对[动态表的连续查询抽象]({{< ref "docs/dev/table/concepts/dynamic_tables" >}}#dynamic-tables-amp-continuous-queries)),所以动态函数在每次查询执行时也会被重新计算(当前实现下等效于每条记录计算)。 + +以下内置函数总是非确定性的(批和流模式下,都在运行时对每条记录进行计算) +- UUID +- RAND +- RAND_INTEGER +- CURRENT_DATABASE +- UNIX_TIMESTAMP +- CURRENT_ROW_TIMESTAMP + +以下内置时间函数是动态的,批处理模式下,将在生成执行计划期间被执行(查询开始),对于流模式,将在运行时对每条记录进行计算 +- CURRENT_DATE +- CURRENT_TIME +- CURRENT_TIMESTAMP +- NOW +- LOCALTIME +- LOCALTIMESTAMP + +注意:`isDynamicFunction` 仅适用于内置函数 + ### 运行时集成 --- diff --git a/docs/content/docs/dev/table/functions/udfs.md b/docs/content/docs/dev/table/functions/udfs.md index 59a38739af3..683a691d768 100644 --- a/docs/content/docs/dev/table/functions/udfs.md +++ b/docs/content/docs/dev/table/functions/udfs.md @@ -605,16 +605,60 @@ the method must return `false`. By default, `isDeterministic()` returns `true`. Furthermore, the `isDeterministic()` method might also influence the runtime behavior. A runtime implementation might be called at two different stages: -**During planning (i.e. pre-flight phase)**: If a function is called with constant expressions +**1. During planning (i.e. pre-flight phase)**: If a function is called with constant expressions or constant expressions can be derived from the given statement, a function is pre-evaluated for constant expression reduction and might not be executed on the cluster anymore. Unless `isDeterministic()` is used to disable constant expression reduction in this case. For example, the following calls to `ABS` are executed during planning: `SELECT ABS(-1) FROM t` and `SELECT ABS(field) FROM t WHERE field = -1`; whereas `SELECT ABS(field) FROM t` is not. -**During runtime (i.e. cluster execution)**: If a function is called with non-constant expressions +**2. During runtime (i.e. cluster execution)**:
[flink] branch master updated: [FLINK-28858][docs] Add document to describe join hints for batch sql
This is an automated email from the ASF dual-hosted git repository. godfrey 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 72f0cb6fc7c [FLINK-28858][docs] Add document to describe join hints for batch sql 72f0cb6fc7c is described below commit 72f0cb6fc7cd635c872de6fcbac9aec4c5beaf69 Author: zhengyunhong.zyh <337361...@qq.com> AuthorDate: Tue Aug 9 16:40:10 2022 +0800 [FLINK-28858][docs] Add document to describe join hints for batch sql This closes #20513 --- .../content.zh/docs/dev/table/sql/queries/hints.md | 247 +++-- docs/content/docs/dev/table/sql/queries/hints.md | 241 +++- docs/static/fig/hint/hint_query_block_union.png| Bin 0 -> 36830 bytes docs/static/fig/hint/hint_query_block_view.png | Bin 0 -> 52391 bytes docs/static/fig/hint/hint_query_block_where.png| Bin 0 -> 35551 bytes 5 files changed, 458 insertions(+), 30 deletions(-) diff --git a/docs/content.zh/docs/dev/table/sql/queries/hints.md b/docs/content.zh/docs/dev/table/sql/queries/hints.md index f3afe1973fd..9e6ce8b3cce 100644 --- a/docs/content.zh/docs/dev/table/sql/queries/hints.md +++ b/docs/content.zh/docs/dev/table/sql/queries/hints.md @@ -24,18 +24,18 @@ specific language governing permissions and limitations under the License. --> -# Hints +# 提示 {{< label Batch >}} {{< label Streaming >}} -SQL hints 是和 SQL 语句一起使用来改变执行计划的。本章介绍如何使用 SQL hints 增强各种方法。 +SQL 提示(SQL Hints)是和 SQL 语句一起使用来改变执行计划的。本章介绍如何使用 SQL 提示来实现各种干预。 -SQL hints 一般可以用于以下: +SQL 提示一般可以用于以下: -- 增强 planner:没有完美的 planner,所以实现 SQL hints 让用户更好地控制执行是非常有意义的; -- 增加元数据(或者统计信息):如"已扫描的表索引"和"一些混洗键(shuffle keys)的倾斜信息"的一些统计数据对于查询来说是动态的,用 hints 来配置它们会非常方便,因为我们从 planner 获得的计划元数据通常不那么准确; -- 算子(Operator)资源约束:在许多情况下,我们会为执行算子提供默认的资源配置,即最小并行度或托管内存(UDF 资源消耗)或特殊资源需求(GPU 或 SSD 磁盘)等,可以使用 SQL hints 非常灵活地为每个查询(非作业)配置资源。 +- 增强 planner:没有完美的 planner,所以实现 SQL 提示让用户更好地控制执行是非常有意义的; +- 增加元数据(或者统计信息):如"已扫描的表索引"和"一些混洗键(shuffle keys)的倾斜信息"的一些统计数据对于查询来说是动态的,用提示来配置它们会非常方便,因为我们从 planner 获得的计划元数据通常不那么准确; +- 算子(Operator)资源约束:在许多情况下,我们会为执行算子提供默认的资源配置,即最小并行度或托管内存(UDF 资源消耗)或特殊资源需求(GPU 或 SSD 磁盘)等,可以使用 SQL 提示非常灵活地为每个查询(非作业)配置资源。 ## 动态表(Dynamic Table)选项 @@ -81,30 +81,169 @@ insert into kafka_table1 /*+ OPTIONS('sink.partitioner'='round-robin') */ select ## 查询提示 +查询提示(`Query Hints`)用于为优化器修改执行计划提供建议,该修改只能在当前查询提示所在的查询块中生效(`Query block`, [什么是查询块](#什么是查询块))。 +目前,Flink 查询提示只支持联接提示(`Join Hints`)。 + +### 语法 +Flink 中的查询提示语法与 Apache Calcite 的语法一致: +```sql +# Query hints: +SELECT /*+ hint [, hint ] */ ... + +hint: +hintName +| hintName '(' optionKey '=' optionVal [, optionKey '=' optionVal ]* ')' +| hintName '(' hintOption [, hintOption ]* ')' + +optionKey: +simpleIdentifier +| stringLiteral + +optionVal: +stringLiteral + +hintOption: +simpleIdentifier +| numericLiteral +| stringLiteral +``` + ### 联接提示 - LOOKUP +联接提示(`Join Hints`)是查询提示(`Query Hints`)的一种,该提示允许用户手动指定表联接(join)时使用的联接策略,来达到优化执行的目的。Flink 联接提示现在支持 `BROADCAST`, +`SHUFFLE_HASH`,`SHUFFLE_MERGE` 和 `NEST_LOOP`。 -{{< label Streaming >}} +{{< hint info >}} +注意: +- 联接提示中定义的表必须存在,否则,将会报表不存在的错误。 +- Flink 联接提示在一个查询块(Query Block)中只支持定义一个提示块,如果定义了多个提示块,类似 `/*+ BROADCAST(t1) */ /*+ SHUFFLE_HASH(t1) */`,则在 SQL 解析时会报错。 +- 在同一个提示块中,Flink 支持在一个联接提示中定义多个表如:`/*+ BROADCAST(t1, t2, ..., tn) */` 或者定义多个联接提示如:`/*+ BROADCAST(t1), BROADCAST(t2), ..., BROADCAST(tn) */`。 +- 对于上述的在一个联接提示中定义多个表或定义多个联接提示的例子,联接提示可能产生冲突。如果冲突产生,Flink 会选择最匹配的表或者联接策略。(详见: [联接提示使用中的冲突](#联接提示使用中的冲突)) + {{< /hint >}} -LOOKUP 联接提示允许用户建议 Flink 优化器: -1. 使用同步或异步的查找函数 -2. 配置异步查找相关参数 -3. 启用延迟重试查找策略 + BROADCAST + +{{< label Batch >}} + +`BROADCAST` 推荐联接使用 `BroadCast` 策略。如果该联接提示生效,不管是否设置了 `table.optimizer.join.broadcast-threshold`, +指定了联接提示的联接端(join side)都会被广播到下游。所以当该联接端是小表时,更推荐使用 `BROADCAST`。 + +{{< hint info >}} +注意: BROADCAST 只支持等值的联接条件,且不支持 Full Outer Join。 +{{< /hint >}} + +# 示例 - 语法 ```sql -SELECT /*+ LOOKUP(hint_options) */ +CREATE TABLE t1 (id BIGINT, name STRING, age INT) WITH (...); +CREATE TABLE t2 (id BIGINT, name STRING, age INT) WITH (...); +CREATE TABLE t3 (id BIGINT, name STRING, age INT) WITH (...); -hint_options: key=value[, key=value]* +-- Flink 会使用 broadcast join,且表 t1 会被当作需 broadcast 的表。 +SELECT /*+ BROADCAST(t1) */ * FROM t1 JOIN t2 ON t1.id = t2.id; -key: -stringLiteral +-- Flink 会在两个联接中都使用 broadcast join,且 t1 和 t3 会被作为需 broadcast 到下游的表。 +SELECT /*+ BROADCAST(t1, t3) */ * FROM t1 JOIN t2 ON t1.id = t2.id JOIN t3 ON t1.id = t3.id; -value: -stringLiteral +-- BROADCAST 只支持等值的联接条件 +-- 联接提示会失效,只能使用支持非等值条件联接的 nested loop join。 +SELECT /*+ BROADCAST(t1) */ * FROM t1 join t2 ON t1.id > t2.id; + +-- BROADCAST 不支持 `Full Outer Join` +-- 联接提示会失效,planner 会根据 cost 选择最合适的联接策略。 +SELECT /*+
[flink] branch master updated: [FLINK-29112][table-planner] Print the lookup join hint on the node in the original RelNode tree for easier debugging
This is an automated email from the ASF dual-hosted git repository. godfrey 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 69872241788 [FLINK-29112][table-planner] Print the lookup join hint on the node in the original RelNode tree for easier debugging 69872241788 is described below commit 69872241788f112cb3b9148269c3c494487d7bc4 Author: lincoln lee AuthorDate: Fri Aug 26 16:30:55 2022 +0800 [FLINK-29112][table-planner] Print the lookup join hint on the node in the original RelNode tree for easier debugging This closes #20686 --- .../planner/plan/utils/RelTreeWriterImpl.scala | 6 ++--- .../plan/stream/sql/join/LookupJoinTest.xml| 28 +++--- 2 files changed, 17 insertions(+), 17 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RelTreeWriterImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RelTreeWriterImpl.scala index 9a7328b91c1..79bfbaf8501 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RelTreeWriterImpl.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RelTreeWriterImpl.scala @@ -22,7 +22,7 @@ import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel import org.apache.calcite.rel.RelNode -import org.apache.calcite.rel.core.{Join, TableScan} +import org.apache.calcite.rel.core.{Correlate, Join, TableScan} import org.apache.calcite.rel.externalize.RelWriterImpl import org.apache.calcite.rel.hint.Hintable import org.apache.calcite.sql.SqlExplainLevel @@ -110,8 +110,8 @@ class RelTreeWriterImpl( if (withJoinHint) { rel match { -case join: Join => - val joinHints = FlinkHints.getAllJoinHints(join.getHints) +case _: Join | _: Correlate => + val joinHints = FlinkHints.getAllJoinHints(rel.asInstanceOf[Hintable].getHints) if (joinHints.nonEmpty) { printValues.add(Pair.of("joinHints", RelExplainUtil.hintsToString(joinHints))) } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/LookupJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/LookupJoinTest.xml index e021cd264c9..b171bda90bf 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/LookupJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/LookupJoinTest.xml @@ -541,7 +541,7 @@ GroupAggregate(groupBy=[b], select=[b, COUNT_RETRACT(a) AS EXPR$1, SUM_RETRACT(c
[flink] branch master updated: [FLINK-28971][docs] Adds user documentation for the new LOOKUP hint
This is an automated email from the ASF dual-hosted git repository. godfrey 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 bbe969059b1 [FLINK-28971][docs] Adds user documentation for the new LOOKUP hint bbe969059b1 is described below commit bbe969059b10f29567d9bed49c10fadeee8d0495 Author: lincoln lee AuthorDate: Mon Aug 15 18:11:47 2022 +0800 [FLINK-28971][docs] Adds user documentation for the new LOOKUP hint This closes #20577 --- .../content.zh/docs/dev/table/sql/queries/hints.md | 285 ++ docs/content/docs/dev/table/sql/queries/hints.md | 320 + 2 files changed, 605 insertions(+) diff --git a/docs/content.zh/docs/dev/table/sql/queries/hints.md b/docs/content.zh/docs/dev/table/sql/queries/hints.md index d6e14e192f9..f3afe1973fd 100644 --- a/docs/content.zh/docs/dev/table/sql/queries/hints.md +++ b/docs/content.zh/docs/dev/table/sql/queries/hints.md @@ -79,4 +79,289 @@ insert into kafka_table1 /*+ OPTIONS('sink.partitioner'='round-robin') */ select ``` +## 查询提示 + +### 联接提示 + + LOOKUP + +{{< label Streaming >}} + +LOOKUP 联接提示允许用户建议 Flink 优化器: +1. 使用同步或异步的查找函数 +2. 配置异步查找相关参数 +3. 启用延迟重试查找策略 + + 语法 +```sql +SELECT /*+ LOOKUP(hint_options) */ + +hint_options: key=value[, key=value]* + +key: +stringLiteral + +value: +stringLiteral +``` + + LOOKUP 提示选项: + + + + + 选项类型 + 选项名称 + 必选 + 选项值类型 + 默认值 + 选项说明 + + + + + table + table + Y + string + N/A + 查找源表的表名 + + + async + async + N + boolean + N/A + 值可以是 'true' 或 'false', 以建议优化器选择对应的查找函数。若底层的连接器无法提供建议模式的查找函数,提示就不会生效 + + + output-mode + N + string + ordered + 值可以是 'ordered' 或 'allow_unordered','allow_unordered' 代表用户允许不保序的输出, 在优化器判断不影响 +正确性的情况下会转成 `AsyncDataStream.OutputMode.UNORDERED`, 否则转成 `ORDERED`。 这与作业参数 +`ExecutionConfigOptions#TABLE_EXEC_ASYNC_LOOKUP_OUTPUT_MODE` 是一致的 + + + capacity + N + integer + 100 + 异步查找使用的底层 `AsyncWaitOperator` 算子的缓冲队列大小 + + + timeout + N + duration + 300s + 异步查找从第一次调用到最终查找完成的超时时间,可能包含了多次重试,在发生 failover 时会重置 + + + retry + retry-predicate + N + string + N/A + 可以是 'lookup_miss',表示在查找结果为空是启用重试 + + + retry-strategy + N + string + N/A + 可以是 'fixed_delay' + + + fixed-delay + N + duration + N/A + 固定延迟策略的延迟时长 + + + max-attempts + N + integer + N/A + 固定延迟策略的最大重试次数 + + + + +注意:其中 +- 'table' 是必选项,需要填写目标联接表的表名(和 FROM 子句引用的表名保持一致),注意当前不支持填写表的别名(这将在后续版本中支持)。 +- 异步查找参数可按需设置一个或多个,未设置的参数按默认值生效。 +- 重试查找参数没有默认值,在需要开启时所有参数都必须设置为有效值。 + + 1. 使用同步或异步的查找函数 +如果连接器同时具备同步和异步查找能力,用户通过给出提示选项值 'async'='false' 来建议优化器选择同步查找, 或 'async'='true' 来建议选择异步查找。 + +示例: +```sql +-- 建议优化器选择同步查找 +LOOKUP('table'='Customers', 'async'='false') + +-- 建议优化器选择异步查找 +LOOKUP('table'='Customers', 'async'='true') +``` +注意:当没有指定 'async' 选项值时,优化器优先选择异步查找,在以下两种情况下优化器会选择同步查找: +1. 当连接器仅实现了同步查找时 +2. 用户在参数 ['table.optimizer.non-deterministic-update.strategy']({{< ref "docs/dev/table/config" >}}#table-optimizer-non-deterministic-update-strategy) 上启用了 'TRY_RESOLVE' 模式,并且优化器推断用户查询中存在非确定性更新的潜在风险时 + + 2. 配置异步查找相关参数 +在异步查找模式下,用户可通过提示选项直接配置异步查找相关参数 + +示例: +```sql +-- 设置异步查找参数 'output-mode', 'capacity', 'timeout', 可按需设置单个或多个参数 +LOOKUP('table'='Customers', 'async'='true', 'output-mode'='allow_unordered', 'capacity'='100', 'timeout'='180s') +``` +注意:联接提示上的异步查找参数和[作业级别配置参数]({{< ref "docs/dev/table/config" >}}#execution-options)的含义是一致的,没有设置的参数值由默认值生效,另一个区别是联接提示作用的范围更小,仅限于当前联接操作中对应联接提示选项设置的表名(未被联接提示作用的其他联接查询不受影响)。 + +例如:作业级别异步查找参数设置为 +```gitexclude +table.exec.async-lookup.output-mode: ORDERED +table.exec.async-lookup.buffer-capacity: 100 +table.exec.async-lookup.timeout: 180s +``` + +那么以下联接提示: +```sql +1. LOOKUP('table'='Customers', 'async'='true', 'output-mode'='allow_unordered') +2. LOOKUP('table'='Customers', 'async'='true', 'timeout'='300s') +``` + +分别等价于: +```sql +1. LOOKUP('table'='Customers', 'async'='true', 'output-mode'='allow_unordered', 'capacity'='100', 'timeout'='180s') +2. LOOKUP('table'='Customers', 'async'='true', 'output-mode'='ordered', 'capacity'='100', 'timeout'='300s') +``` + + 3. 启用延迟重试查找策略 +延迟重试查找希望解决流场景中经常遇到的维表数据更新延迟而不能被流数据正确关联的问题。通过提示选项 'retry-predicate'='lookup_miss' 可设置查找结果为空的重试条件,同时设置重试策略参数来开启重试查找功能(同步或异步查找均可),当前仅支持固定延迟重试策略。 + +固定延迟重试策略参数: +```gitexclude +'retry-strategy'='fixed_delay' +-- 固定重试间隔 +'fixed-delay' +-- 最大重试次数(从重试执行开始计数,比如最大重试次数设置为 1,则对某个具体查找键的一次查找处理实际最多执行 2 次查找请求) +'max-attempts' +``` + +示例: +1. 开启异步查找重试 +```sql +LOOKUP('table'='Customers', 'async'='true', 'retry-predicate'='lookup_miss', 'retry-strategy'='fixed_delay', 'fixed-delay'='10s','max-attempts'='3') +```
[flink] branch master updated (91e1291e942 -> d501b88be55)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 91e1291e942 [FLINK-28976][state] Don't add extra delay to the 1st materialization add d501b88be55 [FLINK-29046][Connectors/Hive] Fix HiveTableSourceStatisticsReportTest fails with Hive 3.x No new revisions were added by this update. Summary of changes: .../table/catalog/hive/client/HiveShimV310.java| 5 +- .../hive/HiveTableSourceStatisticsReportTest.java | 58 -- 2 files changed, 47 insertions(+), 16 deletions(-)
[flink] branch master updated: [FLINK-29059][table-planner] Fix the existing column stats are deleted incorrectly when analyze table for partial columns
This is an automated email from the ASF dual-hosted git repository. godfrey 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 fe392645421 [FLINK-29059][table-planner] Fix the existing column stats are deleted incorrectly when analyze table for partial columns fe392645421 is described below commit fe392645421d10923c75cd5438b91d9ed55900d3 Author: zhengyunhong.zyh <337361...@qq.com> AuthorDate: Wed Aug 24 11:51:29 2022 +0800 [FLINK-29059][table-planner] Fix the existing column stats are deleted incorrectly when analyze table for partial columns This closes #20672 --- .../flink/table/api/internal/AnalyzeTableUtil.java | 32 ++-- .../runtime/batch/sql/AnalyzeTableITCase.java | 92 ++ 2 files changed, 118 insertions(+), 6 deletions(-) diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/AnalyzeTableUtil.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/AnalyzeTableUtil.java index 37471ed52ae..45d324d4e48 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/AnalyzeTableUtil.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/AnalyzeTableUtil.java @@ -85,10 +85,15 @@ public class AnalyzeTableUtil { executeSqlAndGenerateStatistics(tableEnv, columns, statSql); CatalogTableStatistics tableStat = result.f0; catalog.alterPartitionStatistics(objectPath, partitionSpec, tableStat, false); -CatalogColumnStatistics columnStat = result.f1; -if (columnStat != null) { +CatalogColumnStatistics newColumnStat = result.f1; +if (newColumnStat != null) { +CatalogColumnStatistics oldColumnStat = +catalog.getPartitionColumnStatistics(objectPath, partitionSpec); +// merge stats +CatalogColumnStatistics mergedColumnStatistics = +mergeColumnStatistics(oldColumnStat, newColumnStat); catalog.alterPartitionColumnStatistics( -objectPath, partitionSpec, columnStat, false); +objectPath, partitionSpec, mergedColumnStatistics, false); } } } else { @@ -97,14 +102,29 @@ public class AnalyzeTableUtil { executeSqlAndGenerateStatistics(tableEnv, columns, statSql); CatalogTableStatistics tableStat = result.f0; catalog.alterTableStatistics(objectPath, tableStat, false); -CatalogColumnStatistics columnStat = result.f1; -if (columnStat != null) { -catalog.alterTableColumnStatistics(objectPath, columnStat, false); +CatalogColumnStatistics newColumnStat = result.f1; +if (newColumnStat != null) { +CatalogColumnStatistics oldColumnStat = +catalog.getTableColumnStatistics(objectPath); +// merge stats. +CatalogColumnStatistics mergedColumnStatistics = +mergeColumnStatistics(oldColumnStat, newColumnStat); +catalog.alterTableColumnStatistics(objectPath, mergedColumnStatistics, false); } } return TableResultImpl.TABLE_RESULT_OK; } +private static CatalogColumnStatistics mergeColumnStatistics( +CatalogColumnStatistics oldColumnStatistics, +CatalogColumnStatistics newColumnStatistics) { +CatalogColumnStatistics columnStatistics = oldColumnStatistics.copy(); +columnStatistics +.getColumnStatisticsData() +.putAll(newColumnStatistics.getColumnStatisticsData()); +return columnStatistics; +} + private static Tuple2 executeSqlAndGenerateStatistics( TableEnvironmentImpl tableEnv, List columns, String statSql) { diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/AnalyzeTableITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/AnalyzeTableITCase.java index 2ba4e0b4bf0..d61bb8440bd 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/AnalyzeTableITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/AnalyzeTableITCase.java @@ -24,6 +24,7 @@ import org.apache.flink.table.catalog.Catalog; import org.apache.flink.table.catalog.CatalogPartitionImpl; import org.apache.flink.table.catalog.CatalogPartitionSpec; import org.apache.flink.table.catalog.ObjectPath; +
[flink] branch master updated: [FLINK-28493][docs] Add document to describe "ANALYZE TABLE" syntax
This is an automated email from the ASF dual-hosted git repository. godfrey 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 7166625ff78 [FLINK-28493][docs] Add document to describe "ANALYZE TABLE" syntax 7166625ff78 is described below commit 7166625ff787181a59a10f4d75218b4ac87254ed Author: zhengyunhong.zyh <337361...@qq.com> AuthorDate: Tue Aug 9 00:07:00 2022 +0800 [FLINK-28493][docs] Add document to describe "ANALYZE TABLE" syntax This closes #20506 --- docs/content.zh/docs/dev/table/sql/analyze.md | 385 + docs/content.zh/docs/dev/table/sql/overview.md | 3 +- docs/content/docs/dev/table/sql/analyze.md | 382 docs/content/docs/dev/table/sql/overview.md| 3 +- 4 files changed, 771 insertions(+), 2 deletions(-) diff --git a/docs/content.zh/docs/dev/table/sql/analyze.md b/docs/content.zh/docs/dev/table/sql/analyze.md new file mode 100644 index 000..a43573dd96b --- /dev/null +++ b/docs/content.zh/docs/dev/table/sql/analyze.md @@ -0,0 +1,385 @@ +--- +title: "ANALYZE 语句" +weight: 8 +type: docs +aliases: + - /zh/dev/table/sql/analyze.html +--- + + + + +# ANALYZE 语句 + +`ANALYZE` 语句被用于为存在的表收集统计信息,并将统计信息写入该表的 catalog 中。当前版本中,`ANALYZE` 语句只支持 `ANALYZE TABLE`, +且只能由用户手动触发。 + +注意 现在, `ANALYZE TABLE` 只支持批模式(Batch Mode),且只能用于已存在的表, +如果表不存在或者是视图(View)则会报错。 + + + + +## 执行 ANALYZE TABLE 语句 + +{{< tabs "analyze table" >}} +{{< tab "Java" >}} +可以使用 `TableEnvironment` 的 `executeSql()` 方法执行 `ANALYZE TABLE` 语句。 + +以下示例展示了如何在 `TableEnvironment` 中执行一条 `ANALYZE TABLE` 语句。 +{{< /tab >}} +{{< tab "Scala" >}} +可以使用 `TableEnvironment` 的 `executeSql()` 方法执行 `ANALYZE TABLE` 语句。 + +以下示例展示了如何在 `TableEnvironment` 中执行一条 `ANALYZE TABLE` 语句。 +{{< /tab >}} +{{< tab "Python" >}} +可以使用 `TableEnvironment` 的 `execute_sql()` 方法执行 `ANALYZE TABLE` 语句。 + +以下示例展示了如何在 `TableEnvironment` 中执行一条 `ANALYZE TABLE` 语句。 +{{< /tab >}} +{{< tab "SQL CLI" >}} + +`ANALYZE TABLE` 语句可以在 [SQL CLI]({{< ref "docs/dev/table/sqlClient" >}}) 中执行。 + +以下示例展示了如何在 SQL CLI 中执行一条 `ANALYZE TABLE` 语句。 + +{{< /tab >}} +{{< /tabs >}} + +{{< tabs "a5de1760-e363-4b8d-9d6f-0bacb35b9dcf" >}} +{{< tab "Java" >}} +```java +TableEnvironment tableEnv = TableEnvironment.create(...); + +// 注册名为 “Store” 的非分区表 +tableEnv.executeSql( +"CREATE TABLE Store (" + +" `id` BIGINT NOT NULl," + +" `location` VARCHAR(32)," + +" `owner` VARCHAR(32)" + +") with (...)"); + +// 注册名为 “Orders” 的分区表 +tableEnv.executeSql( +"CREATE TABLE Orders (" + +" `id` BIGINT NOT NULl," + +" `product` VARCHAR(32)," + +" `amount` INT," + +" `sold_year` BIGINT", + +" `sold_month` BIGINT", + +" `sold_day` BIGINT" + +") PARTITIONED BY (`sold_year`, `sold_month`, `sold_day`) " +") with (...)"); + +// 非分区表,收集表级别的统计信息(表的统计信息主要为行数(row count))。 +tableEnv.executeSql("ANALYZE TABLE Store COMPUTE STATISTICS"); + +// 非分区表,收集表级别的统计信息和所有列的列统计信息。 +tableEnv.executeSql("ANALYZE TABLE Store COMPUTE STATISTICS FOR ALL COLUMNS"); + +// 非分区表,收集表级别的统计信息和指定列(列: location)的列统计信息。 +tableEnv.executeSql("ANALYZE TABLE Store COMPUTE STATISTICS FOR COLUMNS location"); + + +// 假设分区表 “Orders” 有 4 个分区,分区信息如下: +// Partition1 : (sold_year='2022', sold_month='1', sold_day='10') +// Partition2 : (sold_year='2022', sold_month='1', sold_day='11') +// Partition3 : (sold_year='2022', sold_month='2', sold_day='10') +// Partition4 : (sold_year='2022', sold_month='2', sold_day='11') + + +// 分区表,收集分区 Partition1 的表级别统计信息。 +tableEnv.executeSql("ANALYZE TABLE Orders PARTITION(sold_year='2022', sold_month='1', sold_day='10') COMPUTE STATISTICS"); + +// 分区表,收集分区 Partition1 和 Partition2 的表级别统计信息。 +tableEnv.executeSql("ANALYZE TABLE Orders PARTITION(sold_year='2022', sold_month='1', sold_day) COMPUTE STATISTICS"); + +// 分区表,为所有分区收集表级别统计信息。 +tableEnv.executeSql("ANALYZE TABLE Orders PARTITION(sold_year, sold_month, sold_day) COMPUTE STATISTICS"); + +// 分区表,收集分区 Partition1 的表级别统计信息和所有列的统计信息。 +tableEnv.executeSql("ANALYZE TABLE Orders PARTITION(sold_year='2022', sold_month='1', sold_day='10') COMPUTE STATISTICS FOR ALL COLUMNS"); + +// 分区表,收集分区 Partition1 和 Partition2 的表级别统计信息和所有列统计信息。 +tableEnv.executeSql("ANALYZE TABLE Orders PARTITION(sold_year='2022', sold_month='1', sold_day) COMPUTE STATISTICS FOR ALL COLUMNS"); + +// 分区表,为所有分区收集表级别统计信息和所有列的统计信息。 +tableEnv.executeSql("ANALYZE TABLE Orders PARTITION(sold_year, sold_month, sol
[flink] branch master updated (58c4be49001 -> fcaa4f77e0b)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 58c4be49001 [FLINK-28841][scripts][docs] Document dynamic properties support add fcaa4f77e0b [FLINK-29081][table-planner] Capitalize join hints to avoid case sensitive No new revisions were added by this update. Summary of changes: .../apache/calcite/sql2rel/SqlToRelConverter.java | 3 ++ .../flink/table/planner/hint/FlinkHints.java | 40 ++ .../planner/plan/hints/batch/JoinHintTestBase.java | 22 .../plan/hints/batch/BroadcastJoinHintTest.xml | 21 .../plan/hints/batch/NestLoopJoinHintTest.xml | 21 .../plan/hints/batch/ShuffleHashJoinHintTest.xml | 22 .../plan/hints/batch/ShuffleMergeJoinHintTest.xml | 22 .../optimize/ClearQueryBlockAliasResolverTest.xml | 13 +++ .../planner/plan/optimize/JoinHintResolverTest.xml | 13 +++ 9 files changed, 177 insertions(+)
[flink] 01/02: [FLINK-28993][table-planner] Refactor SupportsDynamicFiltering to provide getAcceptedFilterFields method to avoid modifing the DynamicTableSource object if calling applyDynamicFiltering
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 1a0f591a59b59f0c6ce71f5af9e0660293c33fc1 Author: zhengyunhong.zyh <337361...@qq.com> AuthorDate: Tue Aug 23 11:52:53 2022 +0800 [FLINK-28993][table-planner] Refactor SupportsDynamicFiltering to provide getAcceptedFilterFields method to avoid modifing the DynamicTableSource object if calling applyDynamicFiltering method in join reorder rules This closes #20596 --- .../flink/connectors/hive/HiveTableSource.java | 66 -- .../source/abilities/SupportsDynamicFiltering.java | 18 -- .../batch/DynamicPartitionPruningRule.java | 52 +++-- .../utils/DynamicPartitionPruningUtils.java| 21 +++ .../planner/factories/TestValuesTableFactory.java | 21 ++- 5 files changed, 74 insertions(+), 104 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java index 2570c50b702..edb752ab7e0 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java @@ -40,6 +40,7 @@ import org.apache.flink.orc.util.OrcFormatStatisticsReportUtil; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.ObjectPath; @@ -76,7 +77,6 @@ import javax.annotation.Nullable; import java.time.LocalDateTime; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; @@ -86,8 +86,6 @@ import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.P import static org.apache.flink.connectors.hive.HiveOptions.STREAMING_SOURCE_CONSUME_START_OFFSET; import static org.apache.flink.connectors.hive.HiveOptions.STREAMING_SOURCE_ENABLE; import static org.apache.flink.connectors.hive.util.HivePartitionUtils.getAllPartitions; -import static org.apache.flink.util.Preconditions.checkArgument; -import static org.apache.flink.util.Preconditions.checkState; /** A TableSource implementation to read data from Hive tables. */ public class HiveTableSource @@ -252,47 +250,33 @@ public class HiveTableSource } @Override -public List applyDynamicFiltering(List candidateFilterFields) { -if (catalogTable.getPartitionKeys() != null -&& catalogTable.getPartitionKeys().size() != 0) { -checkArgument( -!candidateFilterFields.isEmpty(), -"At least one field should be provided for dynamic filtering"); - -// only accept partition fields of supported types to do dynamic partition pruning -List dynamicFilterPartitionKeys = new ArrayList<>(); -for (String field : candidateFilterFields) { -if (catalogTable.getPartitionKeys().contains(field) -&& HiveSourceDynamicFileEnumerator.SUPPORTED_TYPES.contains( -catalogTable -.getSchema() -.getFieldDataType(field) -.map(DataType::getLogicalType) -.map(LogicalType::getTypeRoot) -.orElse(null))) { -dynamicFilterPartitionKeys.add(field); -} -} -if (dynamicFilterPartitionKeys.isEmpty()) { -LOG.warn( -"No dynamic filter field is accepted," -+ " only partition fields can use for dynamic filtering."); +public List listAcceptedFilterFields() { +List acceptedFilterFields = new ArrayList<>(); +for (String partitionKey : catalogTable.getPartitionKeys()) { +// Only partition keys with supported types can be returned as accepted filter fields. +if (HiveSourceDynamicFileEnumerator.SUPPORTED_TYPES.contains( +catalogTable +.getSchema() +.getFieldDataType(partitionKey) +.map(DataType::getLogicalType) +.map(Log
[flink] 02/02: [FLINK-28993][table-planner] Fix adjusting join cost for dpp query pattern error
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 1ed1deb351bd7a9d4fbe46dd9e9cf40b08b97fd9 Author: zhengyunhong.zyh <337361...@qq.com> AuthorDate: Tue Aug 23 11:55:39 2022 +0800 [FLINK-28993][table-planner] Fix adjusting join cost for dpp query pattern error This closes #20596 --- .../utils/DynamicPartitionPruningUtils.java| 9 +- .../planner/plan/metadata/FlinkRelMdRowCount.scala | 4 +- .../batch/DynamicPartitionPruningRuleTest.java | 133 ++-- .../batch/DynamicPartitionPruningRuleTest.xml | 171 - 4 files changed, 256 insertions(+), 61 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/DynamicPartitionPruningUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/DynamicPartitionPruningUtils.java index ffad5054d5d..d569772b2e1 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/DynamicPartitionPruningUtils.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/DynamicPartitionPruningUtils.java @@ -171,7 +171,14 @@ public class DynamicPartitionPruningUtils { joinKeys.stream() .map(i -> scan.getRowType().getFieldNames().get(i)) .collect(Collectors.toList()); -factSideFactors.isSuitableFactScanSource = !candidateFields.isEmpty(); +if (candidateFields.isEmpty()) { +factSideFactors.isSuitableFactScanSource = false; +return; +} + +factSideFactors.isSuitableFactScanSource = +!getSuitableDynamicFilteringFieldsInFactSide(tableSource, candidateFields) +.isEmpty(); } else if (rel instanceof HepRelVertex) { visitFactSide(((HepRelVertex) rel).getCurrentRel(), factSideFactors, joinKeys); } else if (rel instanceof Exchange || rel instanceof Filter) { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCount.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCount.scala index cfb3ee08cdd..b20868b1835 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCount.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCount.scala @@ -369,14 +369,14 @@ class FlinkRelMdRowCount private extends MetadataHandler[BuiltInMetadata.RowCoun } else { leftRowCount * selectivityOfNonEquiPred } - return outputRowCount + return outputRowCount * dynamicPartitionPruningFactor } // if joinCondition has no ndv stats and no uniqueKeys stats, // rowCount = (leftRowCount + rightRowCount) * join condition selectivity val crossJoin = copyJoinWithNewCondition(join, rexBuilder.makeLiteral(true)) val selectivity = fmq.getSelectivity(crossJoin, condition) -(leftRowCount + rightRowCount) * selectivity +(leftRowCount + rightRowCount) * selectivity * dynamicPartitionPruningFactor } private def copyJoinWithNewCondition(join: Join, newCondition: RexNode): Join = { diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/physical/batch/DynamicPartitionPruningRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/physical/batch/DynamicPartitionPruningRuleTest.java index 51efc661996..0a7f0c7bc9d 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/physical/batch/DynamicPartitionPruningRuleTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/physical/batch/DynamicPartitionPruningRuleTest.java @@ -22,6 +22,9 @@ import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.config.OptimizerConfigOptions; import org.apache.flink.table.catalog.ObjectPath; import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; +import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataBase; +import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataLong; import org.apache.flink.table.catalog.stats.CatalogTableStatistics; import org.apache.flink.table.planner.factories.TestValuesCatalog; import org.apache.flink.table.planner.utils.BatchTableTestUtil; @@ -30,6 +33,12 @@ import org.apache.flink.table.planner.utils.TableTestBase; import org.junit.Before; import org.junit.Test; +import java.ut
[flink] branch master updated (ee4d27411b3 -> 1ed1deb351b)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from ee4d27411b3 [FLINK-28977] NullPointerException in HybridSourceSplitEnumerator.close (#20587) new 1a0f591a59b [FLINK-28993][table-planner] Refactor SupportsDynamicFiltering to provide getAcceptedFilterFields method to avoid modifing the DynamicTableSource object if calling applyDynamicFiltering method in join reorder rules new 1ed1deb351b [FLINK-28993][table-planner] Fix adjusting join cost for dpp query pattern error 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/connectors/hive/HiveTableSource.java | 66 +++- .../source/abilities/SupportsDynamicFiltering.java | 18 ++- .../batch/DynamicPartitionPruningRule.java | 52 +-- .../utils/DynamicPartitionPruningUtils.java| 30 +++- .../planner/plan/metadata/FlinkRelMdRowCount.scala | 4 +- .../planner/factories/TestValuesTableFactory.java | 21 +-- .../batch/DynamicPartitionPruningRuleTest.java | 133 ++-- .../batch/DynamicPartitionPruningRuleTest.xml | 171 - 8 files changed, 330 insertions(+), 165 deletions(-)
[flink] branch master updated (58296bfd987 -> 2322791284e)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 58296bfd987 [FLINK-29036][docs] Cleanup Source documentation add 2322791284e [FLINK-28995][hive] HiveSourceDynamicFileEnumerator filters out a partition without matching if one of the fields is null but its type is not nullable No new revisions were added by this update. Summary of changes: .../hive/HiveSourceDynamicFileEnumerator.java | 7 +++- .../hive/HiveSourceDynamicFileEnumeratorTest.java | 43 ++ 2 files changed, 49 insertions(+), 1 deletion(-)
[flink] branch master updated (803027a1527 -> b7c6116c98d)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 803027a1527 [FLINK-28676] Update copyright year to 2014-2022 in NOTICE files add b7c6116c98d [FLINK-29012][docs] Fix incorrect description of function minus No new revisions were added by this update. Summary of changes: docs/data/sql_functions.yml| 2 +- docs/data/sql_functions_zh.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-)
[flink] branch master updated (97519d1683f -> 10e4c829021)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 97519d1683f [FLINK-29028][python][docs] Align DataStream.cache in Python DataStream API add 10e4c829021 [FLINK-28992][table-planner] Fix: Change Ndv takes the max value instead of sum of all partitions when getting partition table column stats No new revisions were added by this update. Summary of changes: .../table/catalog/hive/util/HiveStatsUtil.java | 32 -- .../apache/flink/table/plan/stats/ColumnStats.java | 9 +- .../apache/flink/table/plan/stats/TableStats.java | 19 +-- .../flink/table/plan/stats/TableStatsTest.java | 93 --- .../utils/CatalogTableStatisticsConverter.java | 9 +- .../program/FlinkRecomputeStatisticsProgram.java | 14 ++- ...ushPartitionIntoLegacyTableSourceScanRule.scala | 6 +- .../file/table/FileSystemStatisticsReportTest.java | 128 - .../planner/catalog/CatalogStatisticsTest.java | 22 ++-- 9 files changed, 273 insertions(+), 59 deletions(-)
[flink] branch master updated (d524e1dc3fd -> ef97c651f06)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from d524e1dc3fd [FLINK-29009][build] Converge okio new d402fe255cd [FLINK-28987][table-planner] Fix incorrect async to sync lookup fallback path of LegacyTableSourceTable new ef97c651f06 [FLINK-28987][table-planner] Refine description of lookup join transformation with async params and retry strategy for easier debugging 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: .../table/planner/hint/LookupJoinHintOptions.java | 11 +- .../plan/nodes/exec/batch/BatchExecLookupJoin.java |6 +- .../nodes/exec/common/CommonExecLookupJoin.java| 68 +- .../plan/nodes/exec/spec/LookupJoinHintSpec.java | 239 .../nodes/exec/stream/StreamExecLookupJoin.java| 29 +- .../table/planner/plan/utils/LookupJoinUtil.java | 571 + .../physical/batch/BatchPhysicalLookupJoin.scala |4 +- .../physical/common/CommonPhysicalLookupJoin.scala | 58 +- .../physical/stream/StreamPhysicalLookupJoin.scala | 36 +- .../stream/StreamPhysicalLookupJoinRule.scala | 14 +- ...pJoinHintWithInvalidPropagationShuttleTest.java | 10 +- .../nodes/exec/serde/AsyncLookupOptionsTest.java | 138 +++ .../exec/serde/LookupJoinHintSpecSerdeTest.java| 51 - .../nodes/exec/serde/RetryLookupOptionsTest.java | 80 ++ ...ntSpecTest.java => LookupJoinHintTestUtil.java} | 35 +- ...ggAndAllConstantLookupKeyWithTryResolveMode.out |6 +- ...nstantLookupKeyWithTryResolveMode_newSource.out |6 +- .../planner/plan/batch/sql/join/LookupJoinTest.xml | 32 +- .../nodes/exec/operator/BatchOperatorNameTest.xml | 14 +- .../nodes/exec/operator/StreamOperatorNameTest.xml | 14 +- .../testAggAndLeftJoinWithTryResolveMode.out |6 +- .../testJoinTemporalTable.out |6 +- .../testJoinTemporalTableWithAsyncHint.out | 10 +- .../testJoinTemporalTableWithAsyncHint2.out| 10 +- .../testJoinTemporalTableWithAsyncRetryHint.out| 20 +- .../testJoinTemporalTableWithAsyncRetryHint2.out | 20 +- ...testJoinTemporalTableWithProjectionPushDown.out |6 +- .../testJoinTemporalTableWithRetryHint.out | 19 +- .../plan/stream/sql/NonDeterministicDagTest.xml| 40 +- .../table/planner/plan/stream/sql/RankTest.xml |2 +- .../plan/stream/sql/join/LookupJoinTest.xml| 1214 ++-- .../plan/metadata/FlinkRelMdHandlerTestBase.scala |4 +- .../plan/stream/sql/NonDeterministicDagTest.scala |6 +- .../plan/stream/sql/join/LookupJoinTest.scala | 131 ++- .../runtime/stream/sql/AsyncLookupJoinITCase.scala |9 +- 35 files changed, 2029 insertions(+), 896 deletions(-) delete mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/LookupJoinHintSpec.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/AsyncLookupOptionsTest.java delete mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LookupJoinHintSpecSerdeTest.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RetryLookupOptionsTest.java rename flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/spec/{LookupJoinHintSpecTest.java => LookupJoinHintTestUtil.java} (67%)
[flink] 02/02: [FLINK-28987][table-planner] Refine description of lookup join transformation with async params and retry strategy for easier debugging
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit ef97c651f06dc835c36e8213687d66e78d80a0b6 Author: lincoln lee AuthorDate: Thu Aug 18 14:24:41 2022 +0800 [FLINK-28987][table-planner] Refine description of lookup join transformation with async params and retry strategy for easier debugging This closes #20592 --- .../table/planner/plan/utils/LookupJoinUtil.java | 15 + .../physical/common/CommonPhysicalLookupJoin.scala | 3 +- ...ggAndAllConstantLookupKeyWithTryResolveMode.out | 6 +- ...nstantLookupKeyWithTryResolveMode_newSource.out | 6 +- .../planner/plan/batch/sql/join/LookupJoinTest.xml | 32 +- .../nodes/exec/operator/BatchOperatorNameTest.xml | 14 +- .../nodes/exec/operator/StreamOperatorNameTest.xml | 14 +- .../testAggAndLeftJoinWithTryResolveMode.out | 2 +- .../testJoinTemporalTable.out | 2 +- .../testJoinTemporalTableWithAsyncHint.out | 2 +- .../testJoinTemporalTableWithAsyncHint2.out| 2 +- .../testJoinTemporalTableWithAsyncRetryHint.out| 2 +- .../testJoinTemporalTableWithAsyncRetryHint2.out | 2 +- ...testJoinTemporalTableWithProjectionPushDown.out | 2 +- .../testJoinTemporalTableWithRetryHint.out | 2 +- .../plan/stream/sql/NonDeterministicDagTest.xml| 40 +- .../table/planner/plan/stream/sql/RankTest.xml | 2 +- .../plan/stream/sql/join/LookupJoinTest.xml| 986 ++--- .../plan/stream/sql/join/LookupJoinTest.scala | 85 +- 19 files changed, 1027 insertions(+), 192 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/LookupJoinUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/LookupJoinUtil.java index 55399ba9b45..cf4362991f4 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/LookupJoinUtil.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/LookupJoinUtil.java @@ -230,6 +230,11 @@ public final class LookupJoinUtil { public int hashCode() { return Objects.hash(asyncBufferCapacity, asyncTimeout, asyncOutputMode); } + +@Override +public String toString() { +return asyncOutputMode + ", " + asyncTimeout + "ms, " + asyncBufferCapacity; +} } /** RetryOptions includes retry lookup related options. */ @@ -286,6 +291,16 @@ public final class LookupJoinUtil { return Objects.hash(retryPredicate, retryStrategy, retryFixedDelay, retryMaxAttempts); } +@Override +public String toString() { +return retryPredicate ++ ", " ++ retryStrategy ++ ", " ++ retryFixedDelay ++ "ms, " ++ retryMaxAttempts; +} @Nullable public static RetryLookupOptions fromJoinHint(@Nullable RelHint lookupJoinHint) { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalLookupJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalLookupJoin.scala index 36c9211262c..a49b6d590ad 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalLookupJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalLookupJoin.scala @@ -191,7 +191,6 @@ abstract class CommonPhysicalLookupJoin( .explainTerms(pw) .item("table", tableIdentifier.asSummaryString()) .item("joinType", JoinTypeUtil.getFlinkJoinType(joinType)) - .item("async", isAsyncEnabled) .item("lookup", lookupKeys) .itemIf("where", whereString, whereString.nonEmpty) .itemIf( @@ -204,6 +203,8 @@ abstract class CommonPhysicalLookupJoin( remainingCondition.isDefined) .item("select", selection) .itemIf("upsertMaterialize", "true", upsertMaterialize) + .itemIf("async", asyncOptions.getOrElse(""), asyncOptions.isDefined) + .itemIf("retry", retryOptions.getOrElse(""), retryOptions.isDefined) } private def getInputChangelogMode(rel: RelNode): ChangelogMode = rel match { diff --git a/flink-table/flink-table-planner/src/test/resources/explain/stream/join/lookup/testAggAndAllConstantLookupKeyWithTryResolveMode.out b/flink-table/flink-table-planner/src/test/resources/explain/stream/join/
[flink] 01/02: [FLINK-28987][table-planner] Fix incorrect async to sync lookup fallback path of LegacyTableSourceTable
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit d402fe255cdca37568ad6ac585ac6fbdf24b5e74 Author: lincoln lee AuthorDate: Thu Aug 18 13:41:55 2022 +0800 [FLINK-28987][table-planner] Fix incorrect async to sync lookup fallback path of LegacyTableSourceTable by fixing this we can completely avoid create user lookup function instance to determine if async lookup is enabled for execution, and make the exec lookup join node immutable and easier to maintain This closes #20592 --- .../table/planner/hint/LookupJoinHintOptions.java | 11 +- .../plan/nodes/exec/batch/BatchExecLookupJoin.java | 6 +- .../nodes/exec/common/CommonExecLookupJoin.java| 68 ++- .../plan/nodes/exec/spec/LookupJoinHintSpec.java | 239 - .../nodes/exec/stream/StreamExecLookupJoin.java| 29 +- .../table/planner/plan/utils/LookupJoinUtil.java | 556 + .../physical/batch/BatchPhysicalLookupJoin.scala | 4 +- .../physical/common/CommonPhysicalLookupJoin.scala | 55 +- .../physical/stream/StreamPhysicalLookupJoin.scala | 36 +- .../stream/StreamPhysicalLookupJoinRule.scala | 14 +- ...pJoinHintWithInvalidPropagationShuttleTest.java | 10 +- .../nodes/exec/serde/AsyncLookupOptionsTest.java | 138 + .../exec/serde/LookupJoinHintSpecSerdeTest.java| 51 -- .../nodes/exec/serde/RetryLookupOptionsTest.java | 80 +++ ...ntSpecTest.java => LookupJoinHintTestUtil.java} | 35 +- .../testAggAndLeftJoinWithTryResolveMode.out | 4 +- .../testJoinTemporalTable.out | 4 +- .../testJoinTemporalTableWithAsyncHint.out | 8 +- .../testJoinTemporalTableWithAsyncHint2.out| 8 +- .../testJoinTemporalTableWithAsyncRetryHint.out| 18 +- .../testJoinTemporalTableWithAsyncRetryHint2.out | 18 +- ...testJoinTemporalTableWithProjectionPushDown.out | 4 +- .../testJoinTemporalTableWithRetryHint.out | 17 +- .../plan/stream/sql/join/LookupJoinTest.xml| 250 - .../plan/metadata/FlinkRelMdHandlerTestBase.scala | 4 +- .../plan/stream/sql/NonDeterministicDagTest.scala | 6 +- .../plan/stream/sql/join/LookupJoinTest.scala | 46 +- .../runtime/stream/sql/AsyncLookupJoinITCase.scala | 9 +- 28 files changed, 1013 insertions(+), 715 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/LookupJoinHintOptions.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/LookupJoinHintOptions.java index b634572035d..1afb8ff8f9f 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/LookupJoinHintOptions.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/LookupJoinHintOptions.java @@ -53,9 +53,7 @@ public class LookupJoinHintOptions { public static final ConfigOption ASYNC_OUTPUT_MODE = key("output-mode") .enumType(ExecutionConfigOptions.AsyncOutputMode.class) -.defaultValue( - ExecutionConfigOptions.TABLE_EXEC_ASYNC_LOOKUP_OUTPUT_MODE -.defaultValue()) +.noDefaultValue() .withDescription( "Output mode for asynchronous operations which will convert to {@see AsyncDataStream.OutputMode}, ORDERED by default. " + "If set to ALLOW_UNORDERED, will attempt to use {@see AsyncDataStream.OutputMode.UNORDERED} when it does not " @@ -64,17 +62,14 @@ public class LookupJoinHintOptions { public static final ConfigOption ASYNC_CAPACITY = key("capacity") .intType() -.defaultValue( - ExecutionConfigOptions.TABLE_EXEC_ASYNC_LOOKUP_BUFFER_CAPACITY -.defaultValue()) +.noDefaultValue() .withDescription( "The max number of async i/o operation that the async lookup join can trigger."); public static final ConfigOption ASYNC_TIMEOUT = key("timeout") .durationType() -.defaultValue( - ExecutionConfigOptions.TABLE_EXEC_ASYNC_LOOKUP_TIMEOUT.defaultValue()) +.noDefaultValue() .withDescription( "Timeout from first invoke to final completion of asynchronous operation, may include multiple" + " retries, and will be reset in case of failover."); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table
[flink] branch master updated (5d13403429d -> 06e8b7fb30f)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 5d13403429d [FLINK-28800][network] HsFileDataManager should avoid busy-loop when fileReader has not data to read add 06e8b7fb30f [FLINK-28990][table-planner] Fix BatchPhysicalDynamicFilteringDataCollector with empty output type No new revisions were added by this update. Summary of changes: .../batch/DynamicPartitionPruningRule.java | 37 -- .../batch/DynamicPartitionPruningRuleTest.java | 18 +++ .../batch/DynamicPartitionPruningRuleTest.xml | 58 +- 3 files changed, 108 insertions(+), 5 deletions(-)
[flink] branch master updated: [FLINK-28986][table-planner] UNNEST function with nested filter fails to generate plan
This is an automated email from the ASF dual-hosted git repository. godfrey 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 bdffb6bd4ae [FLINK-28986][table-planner] UNNEST function with nested filter fails to generate plan bdffb6bd4ae is described below commit bdffb6bd4ae3ea32bdcd6ec6bce6c6e0e8b92a11 Author: Jane Chan AuthorDate: Tue Aug 16 17:35:54 2022 +0800 [FLINK-28986][table-planner] UNNEST function with nested filter fails to generate plan This closes #20601 --- .../planner/plan/rules/FlinkBatchRuleSets.scala| 2 ++ .../planner/plan/rules/FlinkStreamRuleSets.scala | 2 ++ .../table/planner/plan/batch/sql/UnnestTest.xml| 36 +++ .../plan/rules/logical/LogicalUnnestRuleTest.xml | 42 ++ .../table/planner/plan/stream/sql/UnnestTest.xml | 36 +++ .../table/planner/plan/common/UnnestTestBase.scala | 17 + .../planner/runtime/stream/sql/UnnestITCase.scala | 31 7 files changed, 166 insertions(+) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala index a5b1b197dec..691ca43f339 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala @@ -116,6 +116,8 @@ object FlinkBatchRuleSets { ConvertToNotInOrInRule.INSTANCE, // optimize limit 0 FlinkLimit0RemoveRule.INSTANCE, +// fix: FLINK-28986 nested filter pattern causes unnest rule mismatch +CoreRules.FILTER_MERGE, // unnest rule LogicalUnnestRule.INSTANCE, // Wrap arguments for JSON aggregate functions diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala index d5462682964..97312367287 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala @@ -122,6 +122,8 @@ object FlinkStreamRuleSets { ConvertToNotInOrInRule.INSTANCE, // optimize limit 0 FlinkLimit0RemoveRule.INSTANCE, + // fix: FLINK-28986 nested filter pattern causes unnest rule mismatch + CoreRules.FILTER_MERGE, // unnest rule LogicalUnnestRule.INSTANCE, // rewrite constant table function scan to correlate diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml index 8896c7ee62d..ef3bc2e29d0 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml @@ -291,6 +291,42 @@ LogicalProject(a=[$0], b=[$1], _1=[$2], _2=[$3]) + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml index ddbdb8d61e3..d2e281ba526 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml @@ -308,6 +308,48 @@ LogicalProject(a=[$0], b=[$1], _1=[$2], _2=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]]) +- LogicalTableFunctionScan(invocation=[$UNNEST_ROWS$1($cor0.b)], rowType=[RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2)]) +]]> + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/
[flink] branch master updated (881b2bf046e -> cd173e71162)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 881b2bf046e [FLINK-28861][table] Fix bug in UID format for future migrations and make it configurable add cd173e71162 [FLINK-28899][table-planner] Fix LOOKUP hint with retry option on async lookup mode No new revisions were added by this update. Summary of changes: .../nodes/exec/common/CommonExecLookupJoin.java| 10 +- .../table/planner/plan/utils/LookupJoinUtil.java | 42 +- .../runtime/stream/sql/AsyncLookupJoinITCase.scala | 11 +- .../operators/join/lookup/ResultRetryStrategy.java | 4 +- .../RetryableAsyncLookupFunctionDelegator.java | 102 + .../RetryableAsyncLookupFunctionDelegatorTest.java | 168 + 6 files changed, 318 insertions(+), 19 deletions(-) create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/RetryableAsyncLookupFunctionDelegator.java create mode 100644 flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/RetryableAsyncLookupFunctionDelegatorTest.java
[flink] branch master updated (d0a5023f989 -> c5b5d436843)
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from d0a5023f989 [FLINK-28632][sql-gateway][hive] Allow to GetColumns/GetPrimaryKeys/GetTableTypes in the HiveServer2 Endpoint new 8b25b969d41 [FLINK-28848][table-planner] Introduces LOOKUP join hint to support delayed retry for lookup join (table alias unsupported in hint) new fa6d62dd6bb [hotfix][table-planner] Use scala isInstanceOf to check lookup function type instead of one-level parent class compartion in LookupJoinCodeGenerator new 3a2fc5ef34f [hotfix][runtime] Do last attempt without successfully canceling the retry timer to prevent unexpected incomplete element during finish phase in AsyncWaitOperator new c5b5d436843 [FLINK-28849][table-planner] Fix errors when enable retry on async lookup and add more tests The 4 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: .../api/operators/async/AsyncWaitOperator.java | 8 +- .../table/planner/hint/FlinkHintStrategies.java| 90 ++ .../flink/table/planner/hint/FlinkHints.java | 6 +- .../flink/table/planner/hint/JoinStrategy.java | 11 +- .../table/planner/hint/LookupJoinHintOptions.java | 139 + .../plan/nodes/exec/batch/BatchExecLookupJoin.java | 2 + .../nodes/exec/common/CommonExecLookupJoin.java| 85 +++--- .../plan/nodes/exec/spec/LookupJoinHintSpec.java | 239 .../nodes/exec/stream/StreamExecLookupJoin.java| 5 + .../planner/plan/optimize/JoinHintResolver.java| 20 +- .../table/planner/plan/utils/LookupJoinUtil.java | 312 +++-- .../planner/codegen/LookupJoinCodeGenerator.scala | 5 +- .../physical/common/CommonPhysicalLookupJoin.scala | 11 +- .../physical/stream/StreamPhysicalLookupJoin.scala | 15 +- .../BatchCommonSubGraphBasedOptimizer.scala| 14 +- .../optimize/CommonSubGraphBasedOptimizer.scala| 11 +- .../stream/StreamPhysicalLookupJoinRule.scala | 16 +- ...rJoinHintWithInvalidPropagationShuttleTest.java | 101 +-- ...nHintWithInvalidPropagationShuttleTestBase.java | 128 + ...pJoinHintWithInvalidPropagationShuttleTest.java | 149 ++ .../factories/TestValuesRuntimeFunctions.java | 91 ++ .../planner/factories/TestValuesTableFactory.java | 52 +++- .../batch}/BroadcastJoinHintTest.java | 2 +- .../hints => hints/batch}/JoinHintTestBase.java| 4 +- .../batch}/NestLoopJoinHintTest.java | 2 +- .../batch}/ShuffleHashJoinHintTest.java| 2 +- .../batch}/ShuffleMergeJoinHintTest.java | 2 +- .../exec/serde/LookupJoinHintSpecSerdeTest.java| 51 .../nodes/exec/spec/LookupJoinHintSpecTest.java| 98 +++ .../nodes/exec/stream/LookupJoinJsonPlanTest.java | 68 - .../optimize/ClearQueryBlockAliasResolverTest.java | 2 +- .../plan/optimize/JoinHintResolverTest.java| 2 +- ...upJoinHintWithInvalidPropagationShuttleTest.xml | 127 + .../batch}/BroadcastJoinHintTest.xml | 0 .../hints => hints/batch}/NestLoopJoinHintTest.xml | 0 .../batch}/ShuffleHashJoinHintTest.xml | 0 .../batch}/ShuffleMergeJoinHintTest.xml| 0 .../testJoinTemporalTable.out | 2 +- out => testJoinTemporalTableWithAsyncHint.out} | 10 +- ...out => testJoinTemporalTableWithAsyncHint2.out} | 10 +- ...=> testJoinTemporalTableWithAsyncRetryHint.out} | 14 +- ...> testJoinTemporalTableWithAsyncRetryHint2.out} | 14 +- ...testJoinTemporalTableWithProjectionPushDown.out | 2 +- out => testJoinTemporalTableWithRetryHint.out} | 13 +- .../plan/stream/sql/join/LookupJoinTest.xml| 226 +-- .../plan/metadata/FlinkRelMdHandlerTestBase.scala | 4 +- .../plan/stream/sql/join/LookupJoinTest.scala | 218 +- .../runtime/stream/sql/AsyncLookupJoinITCase.scala | 103 ++- .../runtime/stream/sql/LookupJoinITCase.scala | 105 ++- .../operators/join/lookup/ResultRetryStrategy.java | 69 + .../lookup/RetryableLookupFunctionDelegator.java | 83 ++ .../join/RetryableLookupFunctionDelegatorTest.java | 103 +++ 52 files changed, 2541 insertions(+), 305 deletions(-) create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/LookupJoinHintOptions.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/LookupJoinHintSpec.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttl
[flink] 02/04: [hotfix][table-planner] Use scala isInstanceOf to check lookup function type instead of one-level parent class compartion in LookupJoinCodeGenerator
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit fa6d62dd6bbaa3876656a97ff519e37f9a3f0730 Author: lincoln lee AuthorDate: Tue Aug 9 16:59:00 2022 +0800 [hotfix][table-planner] Use scala isInstanceOf to check lookup function type instead of one-level parent class compartion in LookupJoinCodeGenerator This bug can be reproduced by AsyncLookupJoinITCase#testAsyncJoinTemporalTableWithLookupThresholdWithInsufficientRetry when caching is disabled in FLINK-28849 This closes #20482 --- .../apache/flink/table/planner/codegen/LookupJoinCodeGenerator.scala | 5 + 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/LookupJoinCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/LookupJoinCodeGenerator.scala index bc2fb42b4c4..8f1818af6a5 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/LookupJoinCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/LookupJoinCodeGenerator.scala @@ -250,10 +250,7 @@ object LookupJoinCodeGenerator { val defaultOutputDataType = callContext.getOutputDataType.get() val outputClass = - if ( -udf.getClass.getSuperclass == classOf[LookupFunction] -|| udf.getClass.getSuperclass == classOf[AsyncLookupFunction] - ) { + if (udf.isInstanceOf[LookupFunction] || udf.isInstanceOf[AsyncLookupFunction]) { Some(classOf[RowData]) } else { toScala(extractSimpleGeneric(baseClass, udf.getClass, 0))
[flink] 04/04: [FLINK-28849][table-planner] Fix errors when enable retry on async lookup and add more tests
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit c5b5d4368437fc98b2f5ca31b1f1c6cf3e4ce263 Author: lincoln lee AuthorDate: Tue Aug 9 18:14:08 2022 +0800 [FLINK-28849][table-planner] Fix errors when enable retry on async lookup and add more tests Disable retry on async because of two problems need to be resolved first This closes #20482 --- .../nodes/exec/common/CommonExecLookupJoin.java| 35 +++ .../table/planner/plan/utils/LookupJoinUtil.java | 13 ++- .../physical/common/CommonPhysicalLookupJoin.scala | 7 +- .../factories/TestValuesRuntimeFunctions.java | 91 ++ .../planner/factories/TestValuesTableFactory.java | 52 -- .../plan/stream/sql/join/LookupJoinTest.xml| 2 +- .../runtime/stream/sql/AsyncLookupJoinITCase.scala | 103 +++- .../runtime/stream/sql/LookupJoinITCase.scala | 105 - 8 files changed, 367 insertions(+), 41 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLookupJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLookupJoin.java index 52192ee79ad..d00c888e1fa 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLookupJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLookupJoin.java @@ -542,30 +542,17 @@ public abstract class CommonExecLookupJoin extends ExecNodeBase isLeftOuterJoin, asyncLookupOptions.asyncBufferCapacity); } -/** - * why not implements async-retry directly in AsyncLookupFunction ? - because the active - * sleeping on async callback thread will occupy the task cpu time while the retry support - * in async data stream api provides a more efficient way via processing time service which - * does not occupy callback thread. Both AsyncLookupFunction AsyncTableFunction can support - * retry. does not occupy callback thread. Both AsyncLookupFunction AsyncTableFunction can - * support retry. - */ -if (null != joinHintSpec) { -// simplify code here, not check whether ResultRetryStrategy is NO_RETRY_STRATEGY or not -// because AsyncWaitOperator has short-path optimization during compile time. -return new AsyncWaitOperatorFactory<>( -asyncFunc, -asyncLookupOptions.asyncTimeout, -asyncLookupOptions.asyncBufferCapacity, -convert(asyncLookupOptions.asyncOutputMode), -joinHintSpec.toRetryStrategy()); -} else { -return new AsyncWaitOperatorFactory<>( -asyncFunc, -asyncLookupOptions.asyncTimeout, -asyncLookupOptions.asyncBufferCapacity, -convert(asyncLookupOptions.asyncOutputMode)); -} +// TODO async retry to be supported, can not directly enable retry on 'AsyncWaitOperator' +// because of two reasons: 1. AsyncLookupJoinRunner has a 'stateful' resultFutureBuffer bind +// to each input record (it's non-reenter-able) 2. can not lookup new value if cache empty +// enabled when chained with the new AsyncCachingLookupFunction. This two issues should be +// resolved first before enable async retry. + +return new AsyncWaitOperatorFactory<>( +asyncFunc, +asyncLookupOptions.asyncTimeout, +asyncLookupOptions.asyncBufferCapacity, +convert(asyncLookupOptions.asyncOutputMode)); } private AsyncDataStream.OutputMode convert( diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/LookupJoinUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/LookupJoinUtil.java index 833eb1908a7..529f79b706e 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/LookupJoinUtil.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/LookupJoinUtil.java @@ -400,18 +400,23 @@ public final class LookupJoinUtil { || lookupJoinHintSpec.isAsync(); } -public static boolean isAsyncLookup(RelOptTable temporalTable, Collection lookupKeys) { +public static boolean isAsyncLookup( +RelOptTable temporalTable, +Collection lookupKeys, +LookupJoinHintSpec lookupJoinHintSpec) { +boolean preferA
[flink] 03/04: [hotfix][runtime] Do last attempt without successfully canceling the retry timer to prevent unexpected incomplete element during finish phase in AsyncWaitOperator
This is an automated email from the ASF dual-hosted git repository. godfrey pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 3a2fc5ef34f563c906473cbe4bdd79a9d7eec48e Author: lincoln lee AuthorDate: Tue Aug 9 17:53:04 2022 +0800 [hotfix][runtime] Do last attempt without successfully canceling the retry timer to prevent unexpected incomplete element during finish phase in AsyncWaitOperator It is hard to reproduce this in runtime tests, but occasionally happens in AsyncLookupJoinITCase#testAsyncJoinTemporalTableWithLookupThresholdWithSufficientRetry of FLINK-28849. It's better to add a separate test in runtime. This closes #20482 --- .../flink/streaming/api/operators/async/AsyncWaitOperator.java| 8 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java index ba3f1c3ad87..0d88943b21e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java @@ -359,10 +359,10 @@ public class AsyncWaitOperator if (inFlightDelayRetryHandlers.size() > 0) { for (RetryableResultHandlerDelegator delegator : inFlightDelayRetryHandlers) { assert delegator.delayedRetryTimer != null; -// cancel retry timer, cancel failure means retry action already being executed -if (delegator.delayedRetryTimer.cancel(true)) { -tryOnce(delegator); -} +// fire an attempt intermediately not rely on successfully canceling the retry +// timer for two reasons: 1. cancel retry timer can not be 100% safe 2. there's +// protection for repeated retries +tryOnce(delegator); } inFlightDelayRetryHandlers.clear(); }