From 81f86ae7723179859803b29a70caec0c4db8244f Mon Sep 17 00:00:00 2001 From: Hang Ruan Date: Mon, 10 Nov 2025 10:41:57 +0800 Subject: [PATCH 01/31] Update for 2.2.0 --- docs/config.toml | 19 ++++++++++--------- .../test-scripts/common_docker.sh | 2 +- 2 files changed, 11 insertions(+), 10 deletions(-) diff --git a/docs/config.toml b/docs/config.toml index 6f2014683d4b2..aebb82002e75e 100644 --- a/docs/config.toml +++ b/docs/config.toml @@ -14,7 +14,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -baseURL = '//nightlies.apache.org/flink/flink-docs-master' +baseURL = '//nightlies.apache.org/flink/flink-docs-release-2.2' languageCode = "en-us" title = "Apache Flink" enableGitInfo = false @@ -24,7 +24,7 @@ pygmentsUseClasses = true [params] # Flag whether this is a stable version or not. # Used for the quickstart page. - IsStable = false + IsStable = true # Flag to indicate whether an outdated warning should be shown. ShowOutDatedWarning = false @@ -34,14 +34,14 @@ pygmentsUseClasses = true # we change the version for the complete docs when forking of a release branch # etc. # The full version string as referenced in Maven (e.g. 1.2.1) - Version = "2.2-SNAPSHOT" + Version = "2.2.0" # For stable releases, leave the bugfix version out (e.g. 1.2). For snapshot # release this should be the same as the regular version - VersionTitle = "2.2-SNAPSHOT" + VersionTitle = "2.2" # The branch for this version of Apache Flink - Branch = "master" + Branch = "release-2.2" # The github repository for Apache Flink Repo = "//github.com/apache/flink" @@ -60,19 +60,20 @@ pygmentsUseClasses = true ZhDownloadPage = "//flink.apache.org/zh/downloads.html" - JavaDocs = "//nightlies.apache.org/flink/flink-docs-master/api/java/" + JavaDocs = "//nightlies.apache.org/flink/flink-docs-release-2.2/api/java/" - PyDocs = "//nightlies.apache.org/flink/flink-docs-master/api/python/" + PyDocs = "//nightlies.apache.org/flink/flink-docs-release-2.2/api/python/" # External links at the bottom # of the menu MenuLinks = [ ["Project Homepage", "//flink.apache.org"], - ["JavaDocs", "//nightlies.apache.org/flink/flink-docs-master/api/java/"], - ["PyDocs", "//nightlies.apache.org/flink/flink-docs-master/api/python/"] + ["JavaDocs", "//nightlies.apache.org/flink/flink-docs-release-2.2/api/java/"], + ["PyDocs", "//nightlies.apache.org/flink/flink-docs-release-2.2/api/python/"] ] PreviousDocs = [ + ["2.2", "http://nightlies.apache.org/flink/flink-docs-release-2.2"], ["2.1", "http://nightlies.apache.org/flink/flink-docs-release-2.1"], ["2.0", "http://nightlies.apache.org/flink/flink-docs-release-2.0"], ["1.20", "http://nightlies.apache.org/flink/flink-docs-release-1.20"], diff --git a/flink-end-to-end-tests/test-scripts/common_docker.sh b/flink-end-to-end-tests/test-scripts/common_docker.sh index d4ddbec669b4d..d711b228a7d83 100644 --- a/flink-end-to-end-tests/test-scripts/common_docker.sh +++ b/flink-end-to-end-tests/test-scripts/common_docker.sh @@ -47,7 +47,7 @@ function build_image() { local server_pid=$! echo "Preparing Dockerfiles" - retry_times_with_exponential_backoff 5 git clone https://github.com/apache/flink-docker.git --branch dev-master --single-branch + retry_times_with_exponential_backoff 5 git clone https://github.com/apache/flink-docker.git --branch dev-2.2 --single-branch local java_version=17 if [[ ${PROFILE} == *"jdk17"* ]]; then From b408850714c607fa9ca6edf1d0469168eaa9a4b8 Mon Sep 17 00:00:00 2001 From: Xuyang Date: Mon, 10 Nov 2025 19:28:53 +0800 Subject: [PATCH 02/31] [FLINK-38640][table-planner] Fix NPE in DeltaJoinUtil#isFilterOnOneSetOfUpsertKeys (#27204) (#27215) --- .../planner/plan/utils/DeltaJoinUtil.java | 7 +- .../planner/plan/utils/DeltaJoinUtilTest.java | 83 +++++++++++++++++++ 2 files changed, 88 insertions(+), 2 deletions(-) create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/DeltaJoinUtilTest.java diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/DeltaJoinUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/DeltaJoinUtil.java index e96d5be5324fb..d433ba5ed9df0 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/DeltaJoinUtil.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/DeltaJoinUtil.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.utils; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.table.catalog.Index; import org.apache.flink.table.catalog.ResolvedSchema; @@ -335,12 +336,14 @@ private static boolean areJoinConditionsSupported(StreamPhysicalJoin join) { return isFilterOnOneSetOfUpsertKeys(nonEquiCond.get(), upsertKeys); } - private static boolean isFilterOnOneSetOfUpsertKeys( + @VisibleForTesting + protected static boolean isFilterOnOneSetOfUpsertKeys( RexNode filter, @Nullable Set upsertKeys) { ImmutableBitSet fieldRefIndices = ImmutableBitSet.of( RexNodeExtractor.extractRefInputFields(Collections.singletonList(filter))); - return upsertKeys.stream().anyMatch(uk -> uk.contains(fieldRefIndices)); + return upsertKeys != null + && upsertKeys.stream().anyMatch(uk -> uk.contains(fieldRefIndices)); } private static boolean areAllJoinTableScansSupported(StreamPhysicalJoin join) { diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/DeltaJoinUtilTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/DeltaJoinUtilTest.java new file mode 100644 index 0000000000000..9a5d8fde20227 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/DeltaJoinUtilTest.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.utils; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.planner.calcite.FlinkTypeFactory; +import org.apache.flink.table.planner.calcite.FlinkTypeSystem; +import org.apache.flink.table.types.utils.TypeConversions; + +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.apache.calcite.util.ImmutableBitSet; +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.table.planner.plan.utils.DeltaJoinUtil.isFilterOnOneSetOfUpsertKeys; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link DeltaJoinUtil}. */ +class DeltaJoinUtilTest { + + @Test + void testIsFilterOnOneSetOfUpsertKeys() { + FlinkTypeFactory typeFactory = + new FlinkTypeFactory( + Thread.currentThread().getContextClassLoader(), FlinkTypeSystem.INSTANCE); + // input schema: + // a string, + // b bigint, + // c bigint + List allFieldTypes = + Stream.of(DataTypes.VARCHAR(100), DataTypes.BIGINT(), DataTypes.BIGINT()) + .map(TypeConversions::fromDataToLogicalType) + .map(typeFactory::createFieldTypeFromLogicalType) + .collect(Collectors.toList()); + + RexBuilder rexBuilder = new RexBuilder(typeFactory); + + // a = 'jim' + RexNode filter = + rexBuilder.makeCall( + SqlStdOperatorTable.EQUALS, + rexBuilder.makeInputRef(allFieldTypes.get(0), 0), + rexBuilder.makeLiteral("jim", allFieldTypes.get(0))); + + assertThat(isFilterOnOneSetOfUpsertKeys(filter, Set.of(ImmutableBitSet.of(0)))).isTrue(); + assertThat(isFilterOnOneSetOfUpsertKeys(filter, Set.of(ImmutableBitSet.of(2)))).isFalse(); + assertThat(isFilterOnOneSetOfUpsertKeys(filter, Set.of(ImmutableBitSet.of(0, 1)))).isTrue(); + assertThat(isFilterOnOneSetOfUpsertKeys(filter, Set.of(ImmutableBitSet.of(1, 2)))) + .isFalse(); + assertThat( + isFilterOnOneSetOfUpsertKeys( + filter, Set.of(ImmutableBitSet.of(1), ImmutableBitSet.of(2)))) + .isFalse(); + assertThat( + isFilterOnOneSetOfUpsertKeys( + filter, Set.of(ImmutableBitSet.of(1), ImmutableBitSet.of(0)))) + .isTrue(); + assertThat(isFilterOnOneSetOfUpsertKeys(filter, null)).isFalse(); + } +} From 6e897764bcde6001f000981a862b1bfe476bf351 Mon Sep 17 00:00:00 2001 From: Naci Simsek Date: Wed, 12 Nov 2025 12:39:39 +0100 Subject: [PATCH 03/31] [hotfix] [docs] Ververica URL and Product Name has been updated --- docs/content.zh/docs/deployment/overview.md | 4 ++-- docs/content/docs/deployment/overview.md | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/content.zh/docs/deployment/overview.md b/docs/content.zh/docs/deployment/overview.md index 3fa99b1f18ac3..98313a8add742 100644 --- a/docs/content.zh/docs/deployment/overview.md +++ b/docs/content.zh/docs/deployment/overview.md @@ -302,9 +302,9 @@ Supported Environments: Supported Environment: {{< label Huawei Cloud >}} -#### Ververica Platform +#### Ververica's Unified Streaming Data Platform (Managed Service / BYOC / Self-Managed) -[Website](https://www.ververica.com/platform-overview) +[Website](https://www.ververica.com/product) Supported Environments: {{< label AliCloud >}} diff --git a/docs/content/docs/deployment/overview.md b/docs/content/docs/deployment/overview.md index 6e2fa5695425a..f58d7973983ce 100644 --- a/docs/content/docs/deployment/overview.md +++ b/docs/content/docs/deployment/overview.md @@ -304,9 +304,9 @@ Supported Environments: Supported Environment: {{< label Huawei Cloud >}} -#### Ververica Platform +#### Ververica's Unified Streaming Data Platform (Managed Service / BYOC / Self-Managed) -[Website](https://www.ververica.com/platform) +[Website](https://www.ververica.com/product) Supported Environments: {{< label AliCloud >}} From f9ad1b140e8bac2c7023ed61d19d4ae4c3685b56 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Wed, 12 Nov 2025 22:07:46 +0800 Subject: [PATCH 04/31] [hotfix][examples] Add Python examples on how to read binary data from Kafka --- .../connectors/kafka_avro_format.py | 41 ++++--- .../connectors/kafka_bytearray_format.py | 102 ++++++++++++++++++ .../datastream/connectors/kafka_csv_format.py | 41 ++++--- .../connectors/kafka_json_format.py | 41 ++++--- 4 files changed, 186 insertions(+), 39 deletions(-) create mode 100644 flink-python/pyflink/examples/datastream/connectors/kafka_bytearray_format.py diff --git a/flink-python/pyflink/examples/datastream/connectors/kafka_avro_format.py b/flink-python/pyflink/examples/datastream/connectors/kafka_avro_format.py index 2a66bc4e8b1d7..e6256d85493eb 100644 --- a/flink-python/pyflink/examples/datastream/connectors/kafka_avro_format.py +++ b/flink-python/pyflink/examples/datastream/connectors/kafka_avro_format.py @@ -18,9 +18,10 @@ import logging import sys -from pyflink.common import Types +from pyflink.common import Types, WatermarkStrategy from pyflink.datastream import StreamExecutionEnvironment -from pyflink.datastream.connectors.kafka import FlinkKafkaProducer, FlinkKafkaConsumer +from pyflink.datastream.connectors.kafka import (KafkaRecordSerializationSchema, KafkaSink, + KafkaSource, KafkaOffsetsInitializer) from pyflink.datastream.formats.avro import AvroRowSerializationSchema, AvroRowDeserializationSchema @@ -43,14 +44,20 @@ def write_to_kafka(env): }""" ) - kafka_producer = FlinkKafkaProducer( - topic='test_avro_topic', - serialization_schema=serialization_schema, - producer_config={'bootstrap.servers': 'localhost:9092', 'group.id': 'test_group'} + record_serializer = KafkaRecordSerializationSchema.builder() \ + .set_topic('test_avro_topic') \ + .set_value_serialization_schema(serialization_schema) \ + .build() + kafka_sink = ( + KafkaSink.builder() + .set_record_serializer(record_serializer) + .set_bootstrap_servers('localhost:9092') + .set_property("group.id", "test_group") + .build() ) # note that the output type of ds must be RowTypeInfo - ds.add_sink(kafka_producer) + ds.sink_to(kafka_sink) env.execute() @@ -67,14 +74,22 @@ def read_from_kafka(env): }""" ) - kafka_consumer = FlinkKafkaConsumer( - topics='test_avro_topic', - deserialization_schema=deserialization_schema, - properties={'bootstrap.servers': 'localhost:9092', 'group.id': 'test_group_1'} + kafka_source = ( + KafkaSource.builder() + .set_topics('test_avro_topic') + .set_value_only_deserializer(deserialization_schema) + .set_properties({'bootstrap.servers': 'localhost:9092', 'group.id': 'test_group_1'}) + .set_starting_offsets(KafkaOffsetsInitializer.earliest()) + .build() ) - kafka_consumer.set_start_from_earliest() - env.add_source(kafka_consumer).print() + ds = env.from_source( + kafka_source, + watermark_strategy=WatermarkStrategy.no_watermarks(), + source_name="kafka source" + ) + + ds.print() env.execute() diff --git a/flink-python/pyflink/examples/datastream/connectors/kafka_bytearray_format.py b/flink-python/pyflink/examples/datastream/connectors/kafka_bytearray_format.py new file mode 100644 index 0000000000000..d2e311cd2e658 --- /dev/null +++ b/flink-python/pyflink/examples/datastream/connectors/kafka_bytearray_format.py @@ -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. +################################################################################ +import json +import logging +import sys + +from pyflink.common import Types, ByteArraySchema, WatermarkStrategy +from pyflink.datastream import StreamExecutionEnvironment +from pyflink.datastream.connectors.kafka import KafkaSource, \ + KafkaOffsetsInitializer, KafkaSink, KafkaRecordSerializationSchema + + +# This example works since Flink 2.0 since ByteArraySchema was introduced in Flink 2.0 + +# Make sure that the Kafka cluster is started and the topic 'test_json_topic' is +# created before executing this job. +def write_to_kafka(env): + data = [ + (json.dumps({ + "id": 1, + "country": "USA" + }).encode("utf-8"),), + (json.dumps({ + "id": 2, + "country": "Canada" + }).encode("utf-8"),), + (json.dumps({ + "id": 3, + "country": "Germany" + }).encode("utf-8"),) + ] + type_info = Types.ROW([Types.PRIMITIVE_ARRAY(Types.BYTE())]) + ds = env.from_collection(data, type_info=type_info) + + # declare the output type as Types.PRIMITIVE_ARRAY(Types.BYTE()), + # otherwise, Types.PICKLED_BYTE_ARRAY() will be used by default, it will + # use pickler to serialize the result byte array which is unnecessary + ds = ds.map(lambda x: x[0], output_type=Types.PRIMITIVE_ARRAY(Types.BYTE())) + + record_serializer = KafkaRecordSerializationSchema.builder() \ + .set_topic('test_bytearray_topic') \ + .set_value_serialization_schema(ByteArraySchema()) \ + .build() + kafka_sink = ( + KafkaSink.builder() + .set_record_serializer(record_serializer) + .set_bootstrap_servers('localhost:9092') + .set_property("group.id", "test_group") + .build() + ) + + ds.sink_to(kafka_sink) + env.execute() + + +def read_from_kafka(env): + kafka_source = ( + KafkaSource.builder() + .set_topics('test_bytearray_topic') + .set_value_only_deserializer(ByteArraySchema()) + .set_properties({'bootstrap.servers': 'localhost:9092', 'group.id': 'test_group_1'}) + .set_starting_offsets(KafkaOffsetsInitializer.earliest()) + .build() + ) + + ds = env.from_source( + kafka_source, + watermark_strategy=WatermarkStrategy.no_watermarks(), + source_name="kafka source" + ) + + # the data read out from the source is byte array, decode it as a string + ds.map(lambda data: data.decode("utf-8")).print() + env.execute() + + +if __name__ == '__main__': + logging.basicConfig(stream=sys.stdout, level=logging.INFO, format="%(message)s") + + env = StreamExecutionEnvironment.get_execution_environment() + env.add_jars("file:///path/to/flink-sql-connector-kafka-1.15.0.jar") + + print("start writing data to kafka") + write_to_kafka(env) + + print("start reading data from kafka") + read_from_kafka(env) diff --git a/flink-python/pyflink/examples/datastream/connectors/kafka_csv_format.py b/flink-python/pyflink/examples/datastream/connectors/kafka_csv_format.py index 39c134a8ed336..3fca6240e6660 100644 --- a/flink-python/pyflink/examples/datastream/connectors/kafka_csv_format.py +++ b/flink-python/pyflink/examples/datastream/connectors/kafka_csv_format.py @@ -18,9 +18,10 @@ import logging import sys -from pyflink.common import Types +from pyflink.common import Types, WatermarkStrategy from pyflink.datastream import StreamExecutionEnvironment -from pyflink.datastream.connectors.kafka import FlinkKafkaProducer, FlinkKafkaConsumer +from pyflink.datastream.connectors.kafka import (KafkaRecordSerializationSchema, KafkaSink, + KafkaSource, KafkaOffsetsInitializer) from pyflink.datastream.formats.csv import CsvRowSerializationSchema, CsvRowDeserializationSchema @@ -33,14 +34,20 @@ def write_to_kafka(env): type_info=type_info) serialization_schema = CsvRowSerializationSchema.Builder(type_info).build() - kafka_producer = FlinkKafkaProducer( - topic='test_csv_topic', - serialization_schema=serialization_schema, - producer_config={'bootstrap.servers': 'localhost:9092', 'group.id': 'test_group'} + record_serializer = KafkaRecordSerializationSchema.builder() \ + .set_topic('test_csv_topic') \ + .set_value_serialization_schema(serialization_schema) \ + .build() + kafka_sink = ( + KafkaSink.builder() + .set_record_serializer(record_serializer) + .set_bootstrap_servers('localhost:9092') + .set_property("group.id", "test_group") + .build() ) # note that the output type of ds must be RowTypeInfo - ds.add_sink(kafka_producer) + ds.sink_to(kafka_sink) env.execute() @@ -48,14 +55,22 @@ def read_from_kafka(env): type_info = Types.ROW([Types.INT(), Types.STRING()]) deserialization_schema = CsvRowDeserializationSchema.Builder(type_info).build() - kafka_consumer = FlinkKafkaConsumer( - topics='test_csv_topic', - deserialization_schema=deserialization_schema, - properties={'bootstrap.servers': 'localhost:9092', 'group.id': 'test_group_1'} + kafka_source = ( + KafkaSource.builder() + .set_topics('test_csv_topic') + .set_value_only_deserializer(deserialization_schema) + .set_properties({'bootstrap.servers': 'localhost:9092', 'group.id': 'test_group_1'}) + .set_starting_offsets(KafkaOffsetsInitializer.earliest()) + .build() ) - kafka_consumer.set_start_from_earliest() - env.add_source(kafka_consumer).print() + ds = env.from_source( + kafka_source, + watermark_strategy=WatermarkStrategy.no_watermarks(), + source_name="kafka source" + ) + + ds.print() env.execute() diff --git a/flink-python/pyflink/examples/datastream/connectors/kafka_json_format.py b/flink-python/pyflink/examples/datastream/connectors/kafka_json_format.py index 3cae241ba4316..25480b95cef9c 100644 --- a/flink-python/pyflink/examples/datastream/connectors/kafka_json_format.py +++ b/flink-python/pyflink/examples/datastream/connectors/kafka_json_format.py @@ -18,9 +18,10 @@ import logging import sys -from pyflink.common import Types +from pyflink.common import Types, WatermarkStrategy from pyflink.datastream import StreamExecutionEnvironment -from pyflink.datastream.connectors.kafka import FlinkKafkaProducer, FlinkKafkaConsumer +from pyflink.datastream.connectors.kafka import (KafkaRecordSerializationSchema, KafkaSink, + KafkaSource, KafkaOffsetsInitializer) from pyflink.datastream.formats.json import JsonRowSerializationSchema, JsonRowDeserializationSchema @@ -35,14 +36,20 @@ def write_to_kafka(env): serialization_schema = JsonRowSerializationSchema.Builder() \ .with_type_info(type_info) \ .build() - kafka_producer = FlinkKafkaProducer( - topic='test_json_topic', - serialization_schema=serialization_schema, - producer_config={'bootstrap.servers': 'localhost:9092', 'group.id': 'test_group'} + record_serializer = KafkaRecordSerializationSchema.builder() \ + .set_topic('test_json_topic') \ + .set_value_serialization_schema(serialization_schema) \ + .build() + kafka_sink = ( + KafkaSink.builder() + .set_record_serializer(record_serializer) + .set_bootstrap_servers('localhost:9092') + .set_property("group.id", "test_group") + .build() ) # note that the output type of ds must be RowTypeInfo - ds.add_sink(kafka_producer) + ds.sink_to(kafka_sink) env.execute() @@ -50,14 +57,22 @@ def read_from_kafka(env): deserialization_schema = JsonRowDeserializationSchema.Builder() \ .type_info(Types.ROW([Types.INT(), Types.STRING()])) \ .build() - kafka_consumer = FlinkKafkaConsumer( - topics='test_json_topic', - deserialization_schema=deserialization_schema, - properties={'bootstrap.servers': 'localhost:9092', 'group.id': 'test_group_1'} + kafka_source = ( + KafkaSource.builder() + .set_topics('test_json_topic') + .set_value_only_deserializer(deserialization_schema) + .set_properties({'bootstrap.servers': 'localhost:9092', 'group.id': 'test_group_1'}) + .set_starting_offsets(KafkaOffsetsInitializer.earliest()) + .build() + ) + + ds = env.from_source( + kafka_source, + watermark_strategy=WatermarkStrategy.no_watermarks(), + source_name="kafka source" ) - kafka_consumer.set_start_from_earliest() - env.add_source(kafka_consumer).print() + ds.print() env.execute() From 1bca9b9935afaacad33f7515d4a96a1b38e108bd Mon Sep 17 00:00:00 2001 From: Shengkai <33114724+fsk119@users.noreply.github.com> Date: Tue, 11 Nov 2025 23:15:35 +0800 Subject: [PATCH 05/31] [FLINK-38436][doc] Add vector search doc (#27216) --- .../content.zh/docs/dev/table/sourcesSinks.md | 20 +- .../dev/table/sql/queries/vector-search.md | 116 +++ docs/content/docs/dev/table/sourcesSinks.md | 22 +- .../dev/table/sql/queries/vector-search.md | 116 +++ docs/static/fig/table_connectors.svg | 912 +++++------------- 5 files changed, 537 insertions(+), 649 deletions(-) create mode 100644 docs/content.zh/docs/dev/table/sql/queries/vector-search.md create mode 100644 docs/content/docs/dev/table/sql/queries/vector-search.md diff --git a/docs/content.zh/docs/dev/table/sourcesSinks.md b/docs/content.zh/docs/dev/table/sourcesSinks.md index d4d0ff67b9439..b5cf452ff238a 100644 --- a/docs/content.zh/docs/dev/table/sourcesSinks.md +++ b/docs/content.zh/docs/dev/table/sourcesSinks.md @@ -149,8 +149,9 @@ Flink 会对工厂类逐个进行检查,确保其“标识符”是全局唯 在读取动态表时,表中数据可以是以下情况之一: - changelog 流(支持有界或无界),在 changelog 流结束前,所有的改变都会被源源不断地消费,由 `ScanTableSource` 接口表示。 - 处于一直变换或数据量很大的外部表,其中的数据一般不会被全量读取,除非是在查询某个值时,由 `LookupTableSource` 接口表示。 +- 外部表支持向量搜索,由 `VectorSearchTableSource` 接口表示。 -一个类可以同时实现这两个接口,Planner 会根据查询的 Query 选择相应接口中的方法。 +一个类可以同时实现这三个接口,Planner 会根据查询的 Query 选择相应接口中的方法。 @@ -188,6 +189,23 @@ Flink 会对工厂类逐个进行检查,确保其“标识符”是全局唯 `LookupTableSource` 的实现方法可以是 `TableFunction` 或者 `AsyncTableFunction`,Flink运行时会根据要查询的 key 值,调用这个实现方法进行查询。 +#### Vector Search Table Source + +A `VectorSearchTableSource` searches an external storage system using an input vector and returns the most similar top-K rows during runtime. Users +can determine which algorithm to use to calculate the similarity between the input data and data stored in the external system. In general, most +vector databases support using Euclidean distance or Cosine distance to calculate similarity. + +Compared to `ScanTableSource`, the source does not have to read the entire table and can lazily fetch individual +values from a (possibly continuously changing) external table when necessary. + +Compared to `ScanTableSource`, a `VectorSearchTableSource` currently only supports emitting insert-only changes. + +Compared to `LookupTableSource`, a `VectorSearchTableSource` does not use equality to determine whether a row matches. + +Further abilities are not supported. See the documentation of `org.apache.flink.table.connector.source.VectorSearchTableSource` for more information. + +The runtime implementation of a `VectorSearchTableSource` is a `TableFunction` or `AsyncTableFunction`. The function will be called with the given vector values during runtime. + #### source 端的功能接口 diff --git a/docs/content.zh/docs/dev/table/sql/queries/vector-search.md b/docs/content.zh/docs/dev/table/sql/queries/vector-search.md new file mode 100644 index 0000000000000..4aec4d507eb2d --- /dev/null +++ b/docs/content.zh/docs/dev/table/sql/queries/vector-search.md @@ -0,0 +1,116 @@ +--- +title: "Vector Search" +weight: 7 +type: docs +--- + + +# Vector Search + +{{< label Batch >}} {{< label Streaming >}} + +Flink SQL provides the `VECTOR_SEARCH` table-valued function (TVF) to perform a vector search in SQL queries. This function allows you to search similar rows according to the high-dimension vectors. + +## VECTOR_SEARCH Function + +The `VECTOR_SEARCH` uses a processing-time attribute to correlate rows to the latest version of data in an external table. It's very similar to a lookup join in Flink SQL, however, the difference is +`VECTOR_SEARCH` uses the input data vector to compare the similarity with data in the external table and return the top-k most similar rows. + +### Syntax + +```sql +SELECT * +FROM input_table, LATERAL TABLE(VECTOR_SEARCH( + TABLE vector_table, + input_table.vector_column, + DESCRIPTOR(index_column), + top_k, + [CONFIG => MAP['key', 'value']] + )) +``` + +### Parameters + +- `input_table`: The input table containing the data to be processed +- `vector_table`: The name of external table that allows searching via vector +- `vector_column`: The name of the column in the input table, its type should be FLOAT ARRAY or DOUBLE ARRAY +- `index_column`: A descriptor specifying which column from the vector table should be used to compare the similarity with the input data +- `top_k`: The number of top-k most similar rows to return +- `config`: (Optional) A map of configuration options for the vector search + +### Configuration Options + +The following configuration options can be specified in the config map: + +{{< generated/vector_search_runtime_config_configuration >}} + +### Example + +```sql +-- Basic usage +SELECT * FROM +input_table, LATERAL TABLE(VECTOR_SEARCH( + TABLE vector_table, + input_table.vector_column, + DESCRIPTOR(index_column), + 10 +)); + +-- With configuration options +SELECT * FROM +input_table, LATERAL TABLE(VECTOR_SEARCH( + TABLE vector_table, + input_table.vector_column, + DESCRIPTOR(index_column), + 10, + MAP['async', 'true', 'timeout', '100s'] +)); + +-- Using named parameters +SELECT * FROM +input_table, LATERAL TABLE(VECTOR_SEARCH( + SEARCH_TABLE => TABLE vector_table, + COLUMN_TO_QUERY => input_table.vector_column, + COLUMN_TO_SEARCH => DESCRIPTOR(index_column), + TOP_K => 10, + CONFIG => MAP['async', 'true', 'timeout', '100s'] +)); + +-- Searching with contant value +SELECT * +FROM TABLE(VECTOR_SEARCH( + TABLE vector_table, + ARRAY[10, 20], + DESCRIPTOR(index_column), + 10, +)); +``` + +### Output + +The output table contains all columns from the input table, the vector search table columns and a column named `score` to indicate the similarity between the input row and matched row. + +### Notes + +1. The implementation of the vector table must implement interface `org.apache.flink.table.connector.source.VectorSearchTableSource`. Please refer to [Vector Search Table Source]({{< ref "/docs/dev/table/sourcesSinks" >}}#vector-search-table-source) for details. +2. `VECTOR_SEARCH` only supports to consume append-only tables. +3. `VECTOR_SEARCH` does not require the `LATERAL` keyword when the function call has no correlation with other tables. For example, if the search column is a constant or literal value, `LATERAL` can be omitted. + +{{< top >}} diff --git a/docs/content/docs/dev/table/sourcesSinks.md b/docs/content/docs/dev/table/sourcesSinks.md index 6e8902b45f7ee..a65cabd4b4bb3 100644 --- a/docs/content/docs/dev/table/sourcesSinks.md +++ b/docs/content/docs/dev/table/sourcesSinks.md @@ -174,8 +174,9 @@ When reading a dynamic table, the content can either be considered as: - A continuously changing or very large external table whose content is usually never read entirely but queried for individual values when necessary. This is represented by the `LookupTableSource` interface. +- A table that supports searching via vector. This is represented by the `VectorSearchTableSource` interface. -A class can implement both of these interfaces at the same time. The planner decides about their usage depending +A class can implement all of these interfaces at the same time. The planner decides about their usage depending on the specified query. #### Scan Table Source @@ -223,6 +224,23 @@ for more information. The runtime implementation of a `LookupTableSource` is a `TableFunction` or `AsyncTableFunction`. The function will be called with values for the given lookup keys during runtime. +#### Vector Search Table Source + +A `VectorSearchTableSource` searches an external storage system using an input vector and returns the most similar top-K rows during runtime. Users +can determine which algorithm to use to calculate the similarity between the input data and data stored in the external system. In general, most +vector databases support using Euclidean distance or Cosine distance to calculate similarity. + +Compared to `ScanTableSource`, the source does not have to read the entire table and can lazily fetch individual +values from a (possibly continuously changing) external table when necessary. + +Compared to `ScanTableSource`, a `VectorSearchTableSource` currently only supports emitting insert-only changes. + +Compared to `LookupTableSource`, a `VectorSearchTableSource` does not use equality to determine whether a row matches. + +Further abilities are not supported. See the documentation of `org.apache.flink.table.connector.source.VectorSearchTableSource` for more information. + +The runtime implementation of a `VectorSearchTableSource` is a `TableFunction` or `AsyncTableFunction`. The function will be called with the given vector values during runtime. + #### Source Abilities @@ -282,7 +300,7 @@ will be called with values for the given lookup keys during runtime.
Attention The interfaces above are currently only available for -`ScanTableSource`, not for `LookupTableSource`. +`ScanTableSource`, not for `LookupTableSource` or `VectorSearchTableSource`. ### Dynamic Table Sink diff --git a/docs/content/docs/dev/table/sql/queries/vector-search.md b/docs/content/docs/dev/table/sql/queries/vector-search.md new file mode 100644 index 0000000000000..1ffac76125a6e --- /dev/null +++ b/docs/content/docs/dev/table/sql/queries/vector-search.md @@ -0,0 +1,116 @@ +--- +title: "Vector Search" +weight: 7 +type: docs +--- + + +# Vector Search + +{{< label Batch >}} {{< label Streaming >}} + +Flink SQL provides the `VECTOR_SEARCH` table-valued function (TVF) to perform a vector search in SQL queries. This function allows you to search similar rows according to the high-dimension vectors. + +## VECTOR_SEARCH Function + +The `VECTOR_SEARCH` uses a processing-time attribute to correlate rows to the latest version of data in an external table. It's very similar to a lookup join in Flink SQL, however, the difference is +`VECTOR_SEARCH` uses the input data vector to compare the similarity with data in the external table and return the top-k most similar rows. + +### Syntax + +```sql +SELECT * +FROM input_table, LATERAL TABLE(VECTOR_SEARCH( + TABLE vector_table, + input_table.vector_column, + DESCRIPTOR(index_column), + top_k, + [CONFIG => MAP['key', 'value']] + )) +``` + +### Parameters + +- `input_table`: The input table containing the data to be processed +- `vector_table`: The name of external table that allows searching via vector +- `vector_column`: The name of the column in the input table, its type should be FLOAT ARRAY or DOUBLE ARRAY +- `index_column`: A descriptor specifying which column from the vector table should be used to compare the similarity with the input data +- `top_k`: The number of top-k most similar rows to return +- `config`: (Optional) A map of configuration options for the vector search + +### Configuration Options + +The following configuration options can be specified in the config map: + +{{< generated/vector_search_runtime_config_configuration >}} + +### Example + +```sql +-- Basic usage +SELECT * FROM +input_table, LATERAL TABLE(VECTOR_SEARCH( + TABLE vector_table, + input_table.vector_column, + DESCRIPTOR(index_column), + 10 +)); + +-- With configuration options +SELECT * FROM +input_table, LATERAL TABLE(VECTOR_SEARCH( + TABLE vector_table, + input_table.vector_column, + DESCRIPTOR(index_column), + 10, + MAP['async', 'true', 'timeout', '100s'] +)); + +-- Using named parameters +SELECT * FROM +input_table, LATERAL TABLE(VECTOR_SEARCH( + SEARCH_TABLE => TABLE vector_table, + COLUMN_TO_QUERY => input_table.vector_column, + COLUMN_TO_SEARCH => DESCRIPTOR(index_column), + TOP_K => 10, + CONFIG => MAP['async', 'true', 'timeout', '100s'] +)); + +-- Searching with contant value +SELECT * +FROM TABLE(VECTOR_SEARCH( + TABLE vector_table, + ARRAY[10, 20], + DESCRIPTOR(index_column), + 10, +)); +``` + +### Output + +The output table contains all columns from the input table, the vector search table columns and a column named `score` to indicate the similarity between the input row and matched row. + +### Notes + +1. The implementation of the vector table must implement interface `org.apache.flink.table.connector.source.VectorSearchTableSource`. Please refer to [Vector Search Table Source]({{< ref "/docs/dev/table/sourcesSinks" >}}#vector-search-table-source) for details. +2. `VECTOR_SEARCH` only supports to consume append-only tables. +3. `VECTOR_SEARCH` does not require the `LATERAL` keyword when the function call has no correlation with other tables. For example, if the search column is a constant or literal value, `LATERAL` can be omitted. + +{{< top >}} diff --git a/docs/static/fig/table_connectors.svg b/docs/static/fig/table_connectors.svg index f41d0cb9cfc30..c8c99e8809655 100644 --- a/docs/static/fig/table_connectors.svg +++ b/docs/static/fig/table_connectors.svg @@ -17,650 +17,270 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + From d421622a92d60058d922b6d2215c2a54726addc4 Mon Sep 17 00:00:00 2001 From: Hao Li <1127478+lihaosky@users.noreply.github.com> Date: Thu, 13 Nov 2025 00:00:29 -0800 Subject: [PATCH 06/31] [FLINK-38436][doc] Add Chinese version of vector search doc (#27232) --- .../content.zh/docs/dev/table/sourcesSinks.md | 15 ++--- .../dev/table/sql/queries/vector-search.md | 63 +++++++++---------- 2 files changed, 36 insertions(+), 42 deletions(-) diff --git a/docs/content.zh/docs/dev/table/sourcesSinks.md b/docs/content.zh/docs/dev/table/sourcesSinks.md index b5cf452ff238a..6b932014205d3 100644 --- a/docs/content.zh/docs/dev/table/sourcesSinks.md +++ b/docs/content.zh/docs/dev/table/sourcesSinks.md @@ -191,20 +191,17 @@ Flink 会对工厂类逐个进行检查,确保其“标识符”是全局唯 #### Vector Search Table Source -A `VectorSearchTableSource` searches an external storage system using an input vector and returns the most similar top-K rows during runtime. Users -can determine which algorithm to use to calculate the similarity between the input data and data stored in the external system. In general, most -vector databases support using Euclidean distance or Cosine distance to calculate similarity. +在运行期间, `VectorSearchTableSource` 会使用一个输入向量来搜索外部存储系统,并返回最相似的 Top-K 行。用户可以决定使用何种算法来计算输入数据与外部系统中存储的数据之间的相似度。总的来说,大多数向量数据库支持使用欧几里得距离(Euclidean distance)或余弦距离(Cosine distance)来计算相似度。 -Compared to `ScanTableSource`, the source does not have to read the entire table and can lazily fetch individual -values from a (possibly continuously changing) external table when necessary. +与 `ScanTableSource` 相比,该源无需读取整个表,并可以在需要时从一个(可能在持续变化的)外部表中惰性获取(lazily fetch)单个值。 -Compared to `ScanTableSource`, a `VectorSearchTableSource` currently only supports emitting insert-only changes. +与 `ScanTableSource` 相比,`VectorSearchTableSource` 目前仅支持 insert-only 数据流。 -Compared to `LookupTableSource`, a `VectorSearchTableSource` does not use equality to determine whether a row matches. +与 `LookupTableSource` 相比,`VectorSearchTableSource` 不会使用等值(equality)来判断行是否匹配。 -Further abilities are not supported. See the documentation of `org.apache.flink.table.connector.source.VectorSearchTableSource` for more information. +目前不支持其他更进一步的功能。更多信息请参阅 `org.apache.flink.table.connector.source.VectorSearchTableSource` 的文档。 -The runtime implementation of a `VectorSearchTableSource` is a `TableFunction` or `AsyncTableFunction`. The function will be called with the given vector values during runtime. +`VectorSearchTableSource` 的运行时实现是一个 `TableFunction` 或 `AsyncTableFunction`。在运行时,算子会根据给定的向量值调用该函数。 diff --git a/docs/content.zh/docs/dev/table/sql/queries/vector-search.md b/docs/content.zh/docs/dev/table/sql/queries/vector-search.md index 4aec4d507eb2d..415004cd5dcc2 100644 --- a/docs/content.zh/docs/dev/table/sql/queries/vector-search.md +++ b/docs/content.zh/docs/dev/table/sql/queries/vector-search.md @@ -1,5 +1,5 @@ --- -title: "Vector Search" +title: "向量搜素" weight: 7 type: docs --- @@ -22,22 +22,20 @@ specific language governing permissions and limitations under the License. --> -# Vector Search +# 向量搜索 {{< label Batch >}} {{< label Streaming >}} -Flink SQL provides the `VECTOR_SEARCH` table-valued function (TVF) to perform a vector search in SQL queries. This function allows you to search similar rows according to the high-dimension vectors. +Flink SQL 提供了 `VECTOR_SEARCH` 表值函数 (TVF) 来在 SQL 查询中执行向量搜索。该函数允许您根据高维向量搜索相似的行。 -## VECTOR_SEARCH Function +## VECTOR_SEARCH 函数 -The `VECTOR_SEARCH` uses a processing-time attribute to correlate rows to the latest version of data in an external table. It's very similar to a lookup join in Flink SQL, however, the difference is -`VECTOR_SEARCH` uses the input data vector to compare the similarity with data in the external table and return the top-k most similar rows. +`VECTOR_SEARCH` 使用处理时间属性 (processing-time attribute) 将行与外部表中的最新版本数据关联起来。它与 Flink SQL 中的 lookup-join 非常相似,但区别在于 `VECTOR_SEARCH` 使用输入数据向量与外部表中的数据比较相似度,并返回 top-k 个最相似的行。 -### Syntax +### 语法 ```sql -SELECT * -FROM input_table, LATERAL TABLE(VECTOR_SEARCH( +SELECT * FROM input_table, LATERAL TABLE(VECTOR_SEARCH( TABLE vector_table, input_table.vector_column, DESCRIPTOR(index_column), @@ -46,25 +44,25 @@ FROM input_table, LATERAL TABLE(VECTOR_SEARCH( )) ``` -### Parameters +### 参数 -- `input_table`: The input table containing the data to be processed -- `vector_table`: The name of external table that allows searching via vector -- `vector_column`: The name of the column in the input table, its type should be FLOAT ARRAY or DOUBLE ARRAY -- `index_column`: A descriptor specifying which column from the vector table should be used to compare the similarity with the input data -- `top_k`: The number of top-k most similar rows to return -- `config`: (Optional) A map of configuration options for the vector search +* `input_table`: 包含待处理数据的输入表。 +* `vector_table`: 允许通过向量进行搜索的外部表的名称。 +* `vector_column`: 输入表中的列名,其类型应为 FLOAT ARRAY 或 DOUBLE ARRAY。 +* `index_column`: 一个描述符 (descriptor),指定应使用向量表 (vector_table) 中的哪一列与输入数据进行相似度比较。 +* `top_k`: 要返回的 top-k 个最相似行的数量。 +* `config`: (可选) 用于向量搜索的配置选项。 -### Configuration Options +### 配置选项 -The following configuration options can be specified in the config map: +可以在 config map 中指定以下配置选项: {{< generated/vector_search_runtime_config_configuration >}} -### Example +### 示例 ```sql --- Basic usage +-- 基本用法 SELECT * FROM input_table, LATERAL TABLE(VECTOR_SEARCH( TABLE vector_table, @@ -73,7 +71,7 @@ input_table, LATERAL TABLE(VECTOR_SEARCH( 10 )); --- With configuration options +-- 带配置选项 SELECT * FROM input_table, LATERAL TABLE(VECTOR_SEARCH( TABLE vector_table, @@ -83,7 +81,7 @@ input_table, LATERAL TABLE(VECTOR_SEARCH( MAP['async', 'true', 'timeout', '100s'] )); --- Using named parameters +-- 使用命名参数 SELECT * FROM input_table, LATERAL TABLE(VECTOR_SEARCH( SEARCH_TABLE => TABLE vector_table, @@ -93,24 +91,23 @@ input_table, LATERAL TABLE(VECTOR_SEARCH( CONFIG => MAP['async', 'true', 'timeout', '100s'] )); --- Searching with contant value -SELECT * -FROM TABLE(VECTOR_SEARCH( +-- 使用常量值搜索 +SELECT * FROM TABLE(VECTOR_SEARCH( TABLE vector_table, ARRAY[10, 20], DESCRIPTOR(index_column), - 10, + 10 )); ``` -### Output +### 输出 -The output table contains all columns from the input table, the vector search table columns and a column named `score` to indicate the similarity between the input row and matched row. +输出表包含输入表的所有列、向量搜索表 (vector search table) 的列,以及一个名为 `score` 的列,用于表示输入行与匹配行之间的相似度。 -### Notes +### 注意事项 -1. The implementation of the vector table must implement interface `org.apache.flink.table.connector.source.VectorSearchTableSource`. Please refer to [Vector Search Table Source]({{< ref "/docs/dev/table/sourcesSinks" >}}#vector-search-table-source) for details. -2. `VECTOR_SEARCH` only supports to consume append-only tables. -3. `VECTOR_SEARCH` does not require the `LATERAL` keyword when the function call has no correlation with other tables. For example, if the search column is a constant or literal value, `LATERAL` can be omitted. +1. 向量表 (vector table) 的实现必须实现 `org.apache.flink.table.connector.source.VectorSearchTableSource` 接口。详情请参阅 [Vector Search Table Source]({{< ref "/docs/dev/table/sourcesSinks" >}}#vector-search-table-source)。 +2. `VECTOR_SEARCH` 仅支持读取仅 append-only 表。 +3. 当函数调用与其它表没有关联时,`VECTOR_SEARCH` 不需要 `LATERAL` 关键字。例如,如果搜索列是一个常量或字面值 (literal value),`LATERAL` 可以被省略。 -{{< top >}} +{{< top >}} From 7e2e854e4f7050435106f6996969359b3b9cdd5f Mon Sep 17 00:00:00 2001 From: Yuepeng Pan Date: Thu, 13 Nov 2025 21:26:26 +0800 Subject: [PATCH 07/31] [FLINK-38622][runtime] Enhance the requests and slots balanced allocation logic in DefaultScheduler (#27234) - Introduce ResourceRequestPreMappings to compute the resource matching relationships when allocating all slots in bulk for balanced scheduling of streaming jobs in the default scheduler. - Introduce the test cases for ResourceRequestPreMappings. - Adapt the calculation logic of the TasksBalancedRequestSlotMatchingStrategy for bulk slot allocation using ResourceRequestPreMappings, in order to prevent job scheduling timeouts caused by untimely updates to the relationships between all requests and resources in load-balancing scenarios - Introduce TasksBalancedRequestSlotMatchingStrategyTest for enhancing the TasksBalancedRequestSlotMatchingStrategy testing. (cherry picked from commit b6f0eb6517d1fad186eb654e81aa358e564c6784) --- .../slotpool/ResourceRequestPreMappings.java | 412 ++++++++++++++++++ ...ksBalancedRequestSlotMatchingStrategy.java | 103 +++-- ...cationRequestSlotMatchingStrategyTest.java | 2 +- .../ResourceRequestPreMappingsTest.java | 381 ++++++++++++++++ ...lancedRequestSlotMatchingStrategyTest.java | 114 +++++ 5 files changed, 970 insertions(+), 42 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/ResourceRequestPreMappings.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/ResourceRequestPreMappingsTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/TasksBalancedRequestSlotMatchingStrategyTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/ResourceRequestPreMappings.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/ResourceRequestPreMappings.java new file mode 100644 index 0000000000000..7aec0effe2317 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/ResourceRequestPreMappings.java @@ -0,0 +1,412 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmaster.slotpool; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.util.CollectionUtil; +import org.apache.flink.util.Preconditions; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +/** + * This class is designed to handle the pre-matching of resource requests in the context of balanced + * task scheduling for streaming jobs. During the batch allocation of resources, where resource + * requests are allocated in a single, non-interleaved operation, it is impossible to make immediate + * individual adjustments to unmatched resource requests. This may lead to situations where not all + * resource requests can be successfully fulfilled. For example: + * + *
+ * resource requests:
+ *  - resource request-1: ResourceProfile-1(UNKNOWN)
+ *  - resource request-2: ResourceProfile-2(cpu=2 core, memory=2G)
+ *
+ * available slots:
+ *  - slot-a: ResourceProfile-a(cpu=1 core, memory=1G)
+ *  - slot-b: ResourceProfile-b(cpu=2 core, memory=2G)
+ * 
+ * + * When the strategy {@link TasksBalancedRequestSlotMatchingStrategy} performs resource allocation, + * the following matching mapping might occur, preventing all slot requests from being successfully + * assigned in a consistent manner and thus hindering the scheduling of the entire job: + * + *
+ * the unexpected mapping case:
+ *   - resource request-1: ResourceProfile-1(UNKNOWN) was matched with slot-b: ResourceProfile-b(cpu=2 core, memory=2G)
+ *   - resource request-2: ResourceProfile-2(cpu=2 core, memory=2G) was not matched
+ * 
+ * + * Therefore, it is crucial to determine how ResourceProfiles should match before the batch + * allocation of resource requests, aiming to assure the allocation successfully at least. An ideal + * matching relationship would be: + * + *
+ * - ResourceProfile-1(UNKNOWN)               -> ResourceProfile-a(cpu=1 core, memory=1G)
+ * - ResourceProfile-2(cpu=2 core, memory=2G) -> ResourceProfile-b(cpu=2 core, memory=2G)
+ * 
+ * + * This is the motivation for introducing the current class. + */ +final class ResourceRequestPreMappings { + + private final boolean matchingFulfilled; + // The variable to keep base mappings result related information, which can assure that + // the allocation for all requests could be run successfully at least. + private final Map> + baseRequiredResourcePreMappings; + // The variable to keep the remaining available flexible resources besides the + // baseRequiredResourcePreMappings. + private final Map remainingFlexibleResources; + + private ResourceRequestPreMappings( + boolean matchingFulfilled, + final Map> + baseRequiredResourcePreMappings, + final Map remainingFlexibleResources) { + this.matchingFulfilled = matchingFulfilled; + + this.baseRequiredResourcePreMappings = + CollectionUtil.newHashMapWithExpectedSize(baseRequiredResourcePreMappings.size()); + this.baseRequiredResourcePreMappings.putAll(baseRequiredResourcePreMappings); + + this.remainingFlexibleResources = + CollectionUtil.newHashMapWithExpectedSize(remainingFlexibleResources.size()); + this.remainingFlexibleResources.putAll(remainingFlexibleResources); + } + + static ResourceRequestPreMappings createFrom( + Collection pendingRequests, Collection slots) { + return new ResourceRequestPreMappingsBuilder(pendingRequests, slots).build(); + } + + boolean isMatchingFulfilled() { + return matchingFulfilled; + } + + boolean hasAvailableProfile( + ResourceProfile requiredResourceProfile, ResourceProfile acquirableResourceProfile) { + // Check for base mappings first + Map basePreMapping = + baseRequiredResourcePreMappings.getOrDefault( + requiredResourceProfile, new HashMap<>()); + Integer remainingCnt = basePreMapping.getOrDefault(acquirableResourceProfile, 0); + + if (remainingCnt > 0) { + return true; + } else { + return remainingFlexibleResources.getOrDefault(acquirableResourceProfile, 0) > 0; + } + } + + void decrease( + ResourceProfile requiredResourceProfile, ResourceProfile acquiredResourceProfile) { + Map basePreMapping = + baseRequiredResourcePreMappings.getOrDefault( + requiredResourceProfile, new HashMap<>()); + Integer remainingCntOfBaseMappings = + basePreMapping.getOrDefault(acquiredResourceProfile, 0); + Integer remainingCntOfFlexibleResources = + remainingFlexibleResources.getOrDefault(acquiredResourceProfile, 0); + + Preconditions.checkState( + remainingCntOfBaseMappings > 0 || remainingCntOfFlexibleResources > 0, + "Remaining acquired resource profile %s to match %s is not enough.", + acquiredResourceProfile, + requiredResourceProfile); + + if (remainingCntOfBaseMappings > 0) { + basePreMapping.put(acquiredResourceProfile, remainingCntOfBaseMappings - 1); + return; + } + + if (remainingCntOfFlexibleResources > 0) { + remainingFlexibleResources.put( + acquiredResourceProfile, remainingCntOfFlexibleResources - 1); + // release a resource back to remainingFlexibleResources. + adjustBaseToRemainingFlexibleResources(basePreMapping); + } + } + + private void adjustBaseToRemainingFlexibleResources( + Map basePreMapping) { + Optional> releasableOptOfBaseMappings = + basePreMapping.entrySet().stream() + .filter(entry -> entry.getValue() > 0) + .findFirst(); + Preconditions.checkState( + releasableOptOfBaseMappings.isPresent(), + "No releasable mapping found in the base mappings between resources and requests."); + Map.Entry releasable = releasableOptOfBaseMappings.get(); + ResourceProfile releasableResourceProfile = releasable.getKey(); + + basePreMapping.put(releasableResourceProfile, releasable.getValue() - 1); + + remainingFlexibleResources.compute( + releasableResourceProfile, + (resourceProfile, oldValue) -> oldValue == null ? 1 : oldValue + 1); + } + + @VisibleForTesting + static ResourceRequestPreMappings createFrom( + boolean allMatchable, + final Map> + baseRequiredResourcePreMappings, + final Map remainingFlexibleResources) { + return new ResourceRequestPreMappings( + allMatchable, baseRequiredResourcePreMappings, remainingFlexibleResources); + } + + @VisibleForTesting + Map> getBaseRequiredResourcePreMappings() { + return Collections.unmodifiableMap(baseRequiredResourcePreMappings); + } + + @VisibleForTesting + int getAvailableResourceCntOfBasePreMappings( + ResourceProfile requiredResourceProfile, ResourceProfile acquirableResourceProfile) { + return baseRequiredResourcePreMappings + .getOrDefault(requiredResourceProfile, new HashMap<>()) + .getOrDefault(acquirableResourceProfile, 0); + } + + @VisibleForTesting + Map getRemainingFlexibleResources() { + return Collections.unmodifiableMap(remainingFlexibleResources); + } + + @VisibleForTesting + int getAvailableResourceCntOfRemainingFlexibleMapping( + ResourceProfile availableResourceProfile) { + return remainingFlexibleResources.getOrDefault(availableResourceProfile, 0); + } + + private static final class ResourceRequestPreMappingsBuilder { + + private final Map unfulfilledRequired; + private final Map availableResources; + + // The variable to maintain the base mappings result related information, which can + // assure that the allocation for all requests could be run successfully at least. + private final Map> + baseRequiredResourcePreMappings; + + private ResourceRequestPreMappingsBuilder( + Collection pendingRequests, + Collection slots) { + this.unfulfilledRequired = + pendingRequests.stream() + .collect( + Collectors.groupingBy( + PendingRequest::getResourceProfile, + Collectors.summingInt(ignored -> 1))); + this.unfulfilledRequired + .keySet() + .forEach( + rp -> + Preconditions.checkState( + !rp.equals(ResourceProfile.ZERO) + && !rp.equals(ResourceProfile.ANY), + "The required resource must not be ResourceProfile.ZERO and ResourceProfile.ANY.")); + this.availableResources = + slots.stream() + .collect( + Collectors.groupingBy( + PhysicalSlot::getResourceProfile, + Collectors.summingInt(ignored -> 1))); + this.availableResources + .keySet() + .forEach( + rp -> + Preconditions.checkState( + !rp.equals(ResourceProfile.UNKNOWN) + && !rp.equals(ResourceProfile.ZERO), + "The resource profile of a slot must not be ResourceProfile.UNKNOWN and ResourceProfile.ZERO.")); + this.baseRequiredResourcePreMappings = + CollectionUtil.newHashMapWithExpectedSize(slots.size()); + } + + private ResourceRequestPreMappings build() { + if (unfulfilledRequired.isEmpty() + || availableResources.isEmpty() + || !canFulfillDesiredResources()) { + return currentPreMappings(false); + } + + buildFineGrainedRequestFulfilledExactMapping(); + if (isMatchingFulfilled()) { + return currentPreMappings(true); + } + + buildRemainingFineGrainedRequestFulfilledAnyMapping(); + if (isMatchingFulfilled()) { + return currentPreMappings(true); + } + + buildUnknownRequestFulfilledMapping(); + return currentPreMappings(isMatchingFulfilled()); + } + + private void buildFineGrainedRequestFulfilledExactMapping() { + for (Map.Entry unfulfilledEntry : + new HashMap<>(unfulfilledRequired).entrySet()) { + ResourceProfile requiredFineGrainedResourceProfile = unfulfilledEntry.getKey(); + if (ResourceProfile.UNKNOWN.equals(requiredFineGrainedResourceProfile)) { + continue; + } + // checking fine-grained + int unfulfilledFineGrainedRequiredCnt = unfulfilledEntry.getValue(); + int availableFineGrainedResourceCnt = + availableResources.getOrDefault(requiredFineGrainedResourceProfile, 0); + if (unfulfilledFineGrainedRequiredCnt <= 0 + || availableFineGrainedResourceCnt <= 0) { + continue; + } + + int diff = unfulfilledFineGrainedRequiredCnt - availableFineGrainedResourceCnt; + + Map fulfilledProfileCount = + baseRequiredResourcePreMappings.computeIfAbsent( + requiredFineGrainedResourceProfile, ignored -> new HashMap<>()); + fulfilledProfileCount.put( + requiredFineGrainedResourceProfile, + diff > 0 + ? availableFineGrainedResourceCnt + : unfulfilledFineGrainedRequiredCnt); + + int newUnfulfilledFineGrainedRequiredCnt = Math.max(diff, 0); + int unAvailableFineGrainedResourceCnt = Math.max(-diff, 0); + availableResources.put( + requiredFineGrainedResourceProfile, unAvailableFineGrainedResourceCnt); + unfulfilledRequired.put( + requiredFineGrainedResourceProfile, newUnfulfilledFineGrainedRequiredCnt); + } + } + + private void buildRemainingFineGrainedRequestFulfilledAnyMapping() { + Integer availableResourceProfileANYCount = + availableResources.getOrDefault(ResourceProfile.ANY, 0); + if (availableResourceProfileANYCount <= 0) { + return; + } + + for (Map.Entry unfulfilledEntry : + new HashMap<>(unfulfilledRequired).entrySet()) { + availableResourceProfileANYCount = + availableResources.getOrDefault(ResourceProfile.ANY, 0); + + if (availableResourceProfileANYCount <= 0) { + return; + } + ResourceProfile fineGrainedRequestResourceProfile = unfulfilledEntry.getKey(); + if (ResourceProfile.UNKNOWN.equals(fineGrainedRequestResourceProfile)) { + continue; + } + // checking fine-grained + int unfulfilledFineGrainedRequiredCnt = + unfulfilledRequired.getOrDefault(fineGrainedRequestResourceProfile, 0); + if (unfulfilledFineGrainedRequiredCnt <= 0) { + continue; + } + + int diff = unfulfilledFineGrainedRequiredCnt - availableResourceProfileANYCount; + + Map fulfilledProfileCount = + baseRequiredResourcePreMappings.computeIfAbsent( + fineGrainedRequestResourceProfile, ignored -> new HashMap<>()); + fulfilledProfileCount.put( + ResourceProfile.ANY, + diff > 0 + ? availableResourceProfileANYCount + : unfulfilledFineGrainedRequiredCnt); + + int newUnfulfilledFineGrainedRequiredCnt = Math.max(diff, 0); + int newAvailableResourceProfileANYCount = Math.max(-diff, 0); + availableResources.put(ResourceProfile.ANY, newAvailableResourceProfileANYCount); + unfulfilledRequired.put( + fineGrainedRequestResourceProfile, newUnfulfilledFineGrainedRequiredCnt); + } + } + + private void buildUnknownRequestFulfilledMapping() { + if (unfulfilledRequired.getOrDefault(ResourceProfile.UNKNOWN, 0) <= 0) { + return; + } + + for (Map.Entry availableResourceEntry : + new HashMap<>(availableResources).entrySet()) { + Integer unfulfilledUnknownRequiredCnt = + unfulfilledRequired.getOrDefault(ResourceProfile.UNKNOWN, 0); + ResourceProfile availableResourceProfile = availableResourceEntry.getKey(); + int availableResourceCnt = + availableResources.getOrDefault(availableResourceProfile, 0); + if (availableResourceCnt <= 0) { + continue; + } + if (unfulfilledUnknownRequiredCnt <= 0) { + return; + } + int diff = unfulfilledUnknownRequiredCnt - availableResourceCnt; + Map fulfilledProfileCount = + baseRequiredResourcePreMappings.computeIfAbsent( + ResourceProfile.UNKNOWN, ignored -> new HashMap<>()); + fulfilledProfileCount.put( + availableResourceProfile, + diff > 0 ? availableResourceCnt : unfulfilledUnknownRequiredCnt); + + int newUnfulfilledUnknownRequiredCnt = Math.max(diff, 0); + int newAvailableResourceCnt = Math.max(-diff, 0); + availableResources.put(availableResourceProfile, newAvailableResourceCnt); + unfulfilledRequired.put(ResourceProfile.UNKNOWN, newUnfulfilledUnknownRequiredCnt); + } + } + + private ResourceRequestPreMappings currentPreMappings(boolean matchingFulfilled) { + if (!matchingFulfilled) { + return new ResourceRequestPreMappings(false, new HashMap<>(), new HashMap<>()); + } + return new ResourceRequestPreMappings( + true, + Collections.unmodifiableMap(baseRequiredResourcePreMappings), + Collections.unmodifiableMap(availableResources)); + } + + private boolean isMatchingFulfilled() { + for (ResourceProfile unfulfilledProfile : unfulfilledRequired.keySet()) { + Integer unfulfilled = unfulfilledRequired.getOrDefault(unfulfilledProfile, 0); + if (unfulfilled > 0) { + return false; + } + } + return true; + } + + private boolean canFulfillDesiredResources() { + Integer totalUnfulfilledCnt = + unfulfilledRequired.values().stream().reduce(0, Integer::sum); + Integer totalAvailableCnt = + availableResources.values().stream().reduce(0, Integer::sum); + return totalAvailableCnt >= totalUnfulfilledCnt; + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/TasksBalancedRequestSlotMatchingStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/TasksBalancedRequestSlotMatchingStrategy.java index 8b2fb88c6374d..c70f15ad17e7e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/TasksBalancedRequestSlotMatchingStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/TasksBalancedRequestSlotMatchingStrategy.java @@ -67,12 +67,10 @@ static final class PhysicalSlotElementComparator implements Comparator matchRequestsAndSlots( Collection slots, Collection pendingRequests, Map taskExecutorsLoad) { - if (pendingRequests.isEmpty()) { + ResourceRequestPreMappings resourceRequestPreMappings = + ResourceRequestPreMappings.createFrom(pendingRequests, slots); + if (!resourceRequestPreMappings.isMatchingFulfilled()) { return Collections.emptyList(); } final Collection resultingMatches = new ArrayList<>(); final List sortedRequests = sortByLoadingDescend(pendingRequests); - LOG.debug( - "Available slots: {}, sortedRequests: {}, taskExecutorsLoad: {}", - slots, - sortedRequests, - taskExecutorsLoad); + + logDebugInfo(slots, taskExecutorsLoad, sortedRequests); + Collection slotElements = slots.stream().map(PhysicalSlotElement::new).collect(Collectors.toList()); final Map> profileSlots = @@ -143,33 +149,51 @@ public Collection matchRequestsAndSlots( final Map> taskExecutorSlots = groupSlotsByTaskExecutor(slotElements); for (PendingRequest request : sortedRequests) { - Optional bestSlotEle = - tryMatchPhysicalSlot(request, profileSlots, taskExecutorsLoad); - if (bestSlotEle.isPresent()) { - PhysicalSlotElement slotElement = bestSlotEle.get(); - updateReferenceAfterMatching( - profileSlots, - taskExecutorsLoad, - taskExecutorSlots, - slotElement, - request.getLoading()); + ResourceProfile requestProfile = request.getResourceProfile(); + Optional bestSlotEleOpt = + tryMatchPhysicalSlot( + request, profileSlots, taskExecutorsLoad, resourceRequestPreMappings); + if (bestSlotEleOpt.isPresent()) { + PhysicalSlotElement slotElement = bestSlotEleOpt.get(); + updateTaskExecutorsLoad(taskExecutorsLoad, request, slotElement); + updateReferenceRemainingSlots(profileSlots, taskExecutorSlots, slotElement); + resourceRequestPreMappings.decrease( + requestProfile, slotElement.getResourceProfile()); resultingMatches.add(RequestSlotMatch.createFor(request, slotElement.physicalSlot)); } } return resultingMatches; } + private static void updateTaskExecutorsLoad( + Map taskExecutorsLoad, + PendingRequest request, + PhysicalSlotElement slotElement) { + taskExecutorsLoad.compute( + slotElement.getResourceID(), + (ignoredId, oldLoading) -> + Objects.isNull(oldLoading) + ? request.getLoading() + : oldLoading.merge(request.getLoading())); + } + + private static void logDebugInfo( + Collection slots, + Map taskExecutorsLoad, + List sortedRequests) { + LOG.debug( + "Available slots: {}, sortedRequests: {}, taskExecutorsLoad: {}", + slots, + sortedRequests, + taskExecutorsLoad); + } + private Map> groupSlotsByTaskExecutor( Collection slotElements) { return slotElements.stream() .collect( Collectors.groupingBy( - physicalSlot -> - physicalSlot - .physicalSlot - .getTaskManagerLocation() - .getResourceID(), - Collectors.toSet())); + PhysicalSlotElement::getResourceID, Collectors.toSet())); } private Map> getSlotCandidatesByProfile( @@ -180,7 +204,7 @@ private Map> getSlotCand new PhysicalSlotElementPriorityComparator(taskExecutorsLoad); for (PhysicalSlotElement slotEle : slotElements) { result.compute( - slotEle.physicalSlot.getResourceProfile(), + slotEle.getResourceProfile(), (resourceProfile, oldSlots) -> { HeapPriorityQueue values = Objects.isNull(oldSlots) @@ -197,12 +221,17 @@ private Map> getSlotCand private Optional tryMatchPhysicalSlot( PendingRequest request, Map> profileToSlotMap, - Map taskExecutorsLoad) { + Map taskExecutorsLoad, + ResourceRequestPreMappings resourceRequestPreMappings) { final ResourceProfile requestProfile = request.getResourceProfile(); final Set candidateProfiles = profileToSlotMap.keySet().stream() - .filter(slotProfile -> slotProfile.isMatching(requestProfile)) + .filter( + slotProfile -> + slotProfile.isMatching(requestProfile) + && resourceRequestPreMappings.hasAvailableProfile( + requestProfile, slotProfile)) .collect(Collectors.toSet()); return candidateProfiles.stream() @@ -216,25 +245,17 @@ private Optional tryMatchPhysicalSlot( .min(new PhysicalSlotElementComparator(taskExecutorsLoad)); } - private void updateReferenceAfterMatching( + private void updateReferenceRemainingSlots( Map> profileSlots, - Map taskExecutorsLoad, Map> taskExecutorSlots, - PhysicalSlotElement targetSlotElement, - LoadingWeight loading) { - final ResourceID tmID = - targetSlotElement.physicalSlot.getTaskManagerLocation().getResourceID(); - // Update the loading for the target task executor. - taskExecutorsLoad.compute( - tmID, - (ignoredId, oldLoading) -> - Objects.isNull(oldLoading) ? loading : oldLoading.merge(loading)); + PhysicalSlotElement targetSlotElement) { + final ResourceID tmID = targetSlotElement.getResourceID(); // Update the sorted set for slots that is located on the same task executor as targetSlot. // Use Map#remove to avoid the ConcurrentModifyException. final Set slotToReSort = taskExecutorSlots.remove(tmID); for (PhysicalSlotElement slotEle : slotToReSort) { HeapPriorityQueue slotsOfProfile = - profileSlots.get(slotEle.physicalSlot.getResourceProfile()); + profileSlots.get(slotEle.getResourceProfile()); // Re-add for the latest order. slotsOfProfile.remove(slotEle); if (!slotEle.equals(targetSlotElement)) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/PreferredAllocationRequestSlotMatchingStrategyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/PreferredAllocationRequestSlotMatchingStrategyTest.java index cbb8d2cda4a10..3a047d8b45bbf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/PreferredAllocationRequestSlotMatchingStrategyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/PreferredAllocationRequestSlotMatchingStrategyTest.java @@ -221,7 +221,7 @@ private static TestingPhysicalSlot createSlotAndGrainProfile(TaskManagerLocation return createSlot(finedGrainProfile, new AllocationID(), tmLocation); } - private static TestingPhysicalSlot createSlot( + static TestingPhysicalSlot createSlot( ResourceProfile profile, AllocationID allocationId, TaskManagerLocation tmLocation) { return TestingPhysicalSlot.builder() .withAllocationID(allocationId) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/ResourceRequestPreMappingsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/ResourceRequestPreMappingsTest.java new file mode 100644 index 0000000000000..c2bf3fd691b5f --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/ResourceRequestPreMappingsTest.java @@ -0,0 +1,381 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.runtime.jobmaster.slotpool; + +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.jobmaster.SlotRequestId; +import org.apache.flink.runtime.scheduler.adaptive.allocator.TestingSlot; +import org.apache.flink.runtime.scheduler.loading.DefaultLoadingWeight; +import org.apache.flink.util.Preconditions; + +import org.junit.jupiter.api.Test; + +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Test for {@link ResourceRequestPreMappings}. */ +class ResourceRequestPreMappingsTest { + + private static final ResourceProfile smallFineGrainedResourceProfile = + ResourceProfile.newBuilder().setManagedMemoryMB(10).build(); + + private static final ResourceProfile bigGrainedResourceProfile = + ResourceProfile.newBuilder().setManagedMemoryMB(20).build(); + + @Test + void testIncludeInvalidProfileOfRequestOrResource() { + // For invalid resource. + ResourceProfile[] profiles = + new ResourceProfile[] {ResourceProfile.UNKNOWN, ResourceProfile.ZERO}; + for (ResourceProfile profile : profiles) { + assertThatThrownBy( + () -> + ResourceRequestPreMappings.createFrom( + Collections.emptyList(), newTestingSlots(profile))) + .isInstanceOf(IllegalStateException.class); + } + + // For invalid request. + profiles = new ResourceProfile[] {ResourceProfile.ANY, ResourceProfile.ZERO}; + for (ResourceProfile profile : profiles) { + assertThatThrownBy( + () -> + ResourceRequestPreMappings.createFrom( + newPendingRequests(profile), Collections.emptyList())) + .isInstanceOf(IllegalStateException.class); + } + } + + @Test + void testBuildWhenUnavailableTotalResourcesOrEmptyRequestsResources() { + // Testing for unavailable total resource + ResourceRequestPreMappings preMappings = + ResourceRequestPreMappings.createFrom( + newPendingRequests(ResourceProfile.UNKNOWN), Collections.emptyList()); + assertThat(preMappings.isMatchingFulfilled()).isFalse(); + assertThat(preMappings.getBaseRequiredResourcePreMappings()).isEmpty(); + + // Testing for empty slots or requests + preMappings = + ResourceRequestPreMappings.createFrom( + Collections.emptyList(), Collections.emptyList()); + assertNotMatchable(preMappings); + } + + @Test + void testBuildWhenMissingResourceToMatchFineGrainedRequest() { + + // Testing for missing available fine-grained resources when only fine-grained request + ResourceRequestPreMappings preMappings = + ResourceRequestPreMappings.createFrom( + newPendingRequests( + smallFineGrainedResourceProfile, + smallFineGrainedResourceProfile, + bigGrainedResourceProfile), + newTestingSlots( + smallFineGrainedResourceProfile, + smallFineGrainedResourceProfile, + smallFineGrainedResourceProfile)); + assertNotMatchable(preMappings); + + // Testing for missing available fine-grained resources when fine-grained and unknown + // requests. + preMappings = + ResourceRequestPreMappings.createFrom( + newPendingRequests( + ResourceProfile.UNKNOWN, + smallFineGrainedResourceProfile, + bigGrainedResourceProfile), + newTestingSlots( + smallFineGrainedResourceProfile, + smallFineGrainedResourceProfile, + smallFineGrainedResourceProfile)); + assertNotMatchable(preMappings); + } + + @Test + void testBuildSuccessfullyThatFinedGrainedMatchedExactly() { + ResourceRequestPreMappings preMappings = + ResourceRequestPreMappings.createFrom( + newPendingRequests( + smallFineGrainedResourceProfile, + smallFineGrainedResourceProfile, + bigGrainedResourceProfile), + newTestingSlots( + bigGrainedResourceProfile, + smallFineGrainedResourceProfile, + smallFineGrainedResourceProfile, + smallFineGrainedResourceProfile)); + assertThat(preMappings.isMatchingFulfilled()).isTrue(); + assertThat(preMappings.getBaseRequiredResourcePreMappings()) + .hasSize(2) + .contains( + new AbstractMap.SimpleEntry<>( + smallFineGrainedResourceProfile, + new HashMap<>() { + { + put(smallFineGrainedResourceProfile, 2); + } + }), + new AbstractMap.SimpleEntry<>( + bigGrainedResourceProfile, + new HashMap<>() { + { + put(bigGrainedResourceProfile, 1); + } + })); + assertThat(preMappings.getRemainingFlexibleResources()) + .contains(new AbstractMap.SimpleEntry<>(smallFineGrainedResourceProfile, 1)); + } + + @Test + void testBuildSuccessfullyThatFinedGrainedToMatchedUnknownRequests() { + + // Testing for available all resources and no UNKNOWN required resource. + ResourceRequestPreMappings preMappings = + ResourceRequestPreMappings.createFrom( + newPendingRequests( + ResourceProfile.UNKNOWN, + ResourceProfile.UNKNOWN, + smallFineGrainedResourceProfile, + bigGrainedResourceProfile), + newTestingSlots( + bigGrainedResourceProfile, + bigGrainedResourceProfile, + bigGrainedResourceProfile, + ResourceProfile.ANY, + smallFineGrainedResourceProfile, + smallFineGrainedResourceProfile)); + assertThat(preMappings.isMatchingFulfilled()).isTrue(); + assertThat(preMappings.getBaseRequiredResourcePreMappings()) + .hasSize(3) + .contains( + new AbstractMap.SimpleEntry<>( + smallFineGrainedResourceProfile, + new HashMap<>() { + { + put(smallFineGrainedResourceProfile, 1); + } + }), + new AbstractMap.SimpleEntry<>( + bigGrainedResourceProfile, + new HashMap<>() { + { + put(bigGrainedResourceProfile, 1); + } + })); + Map unknownBaseMapping = + preMappings.getBaseRequiredResourcePreMappings().get(ResourceProfile.UNKNOWN); + assertThat(unknownBaseMapping.values().stream().reduce(0, Integer::sum)).isEqualTo(2); + assertThat( + preMappings.getRemainingFlexibleResources().values().stream() + .reduce(0, Integer::sum)) + .isEqualTo(2); + } + + @Test + void testBuildSuccessfullyThatAnyToMatchedUnknownAndFineGrainedRequests() { + + // Testing for available all resources and no UNKNOWN required resource. + ResourceRequestPreMappings preMappings = + ResourceRequestPreMappings.createFrom( + newPendingRequests( + ResourceProfile.UNKNOWN, + ResourceProfile.UNKNOWN, + smallFineGrainedResourceProfile, + smallFineGrainedResourceProfile, + bigGrainedResourceProfile, + bigGrainedResourceProfile), + newTestingSlots( + bigGrainedResourceProfile, + smallFineGrainedResourceProfile, + ResourceProfile.ANY, + ResourceProfile.ANY, + ResourceProfile.ANY, + ResourceProfile.ANY)); + assertThat(preMappings.isMatchingFulfilled()).isTrue(); + assertThat(preMappings.getBaseRequiredResourcePreMappings()) + .hasSize(3) + .contains( + new AbstractMap.SimpleEntry<>( + smallFineGrainedResourceProfile, + new HashMap<>() { + { + put(smallFineGrainedResourceProfile, 1); + put(ResourceProfile.ANY, 1); + } + }), + new AbstractMap.SimpleEntry<>( + bigGrainedResourceProfile, + new HashMap<>() { + { + put(bigGrainedResourceProfile, 1); + put(ResourceProfile.ANY, 1); + } + }), + new AbstractMap.SimpleEntry<>( + ResourceProfile.UNKNOWN, + new HashMap<>() { + { + put(ResourceProfile.ANY, 2); + } + })); + assertThat( + preMappings.getRemainingFlexibleResources().values().stream() + .reduce(0, Integer::sum)) + .isZero(); + } + + @Test + void testHasAvailableProfile() { + ResourceRequestPreMappings mappings = + ResourceRequestPreMappings.createFrom( + newPendingRequests(ResourceProfile.UNKNOWN, ResourceProfile.UNKNOWN), + newTestingSlots( + smallFineGrainedResourceProfile, + smallFineGrainedResourceProfile, + smallFineGrainedResourceProfile)); + + // Testing available resource in flexible resources + assertThat( + mappings.hasAvailableProfile( + smallFineGrainedResourceProfile, smallFineGrainedResourceProfile)) + .isTrue(); + assertThat( + mappings.hasAvailableProfile( + smallFineGrainedResourceProfile, bigGrainedResourceProfile)) + .isFalse(); + + // Testing available resource in base mapping resources + assertThat( + mappings.hasAvailableProfile( + ResourceProfile.UNKNOWN, smallFineGrainedResourceProfile)) + .isTrue(); + assertThat(mappings.hasAvailableProfile(ResourceProfile.UNKNOWN, bigGrainedResourceProfile)) + .isFalse(); + } + + @Test + void testDecrease() { + // Testing decrease resource in base mapping + ResourceRequestPreMappings mappings = + ResourceRequestPreMappings.createFrom( + newPendingRequests(ResourceProfile.UNKNOWN, ResourceProfile.UNKNOWN), + newTestingSlots( + smallFineGrainedResourceProfile, + smallFineGrainedResourceProfile, + smallFineGrainedResourceProfile)); + + // Testing decrease resource in base mapping resources successfully + mappings.decrease(ResourceProfile.UNKNOWN, smallFineGrainedResourceProfile); + assertThat( + mappings.getAvailableResourceCntOfBasePreMappings( + ResourceProfile.UNKNOWN, smallFineGrainedResourceProfile)) + .isOne(); + // Testing decrease resource in base mapping resources failed + assertThatThrownBy( + () -> + mappings.decrease( + smallFineGrainedResourceProfile, + smallFineGrainedResourceProfile)) + .isInstanceOf(IllegalStateException.class); + + // Testing decrease resource in flexible resources + ResourceRequestPreMappings mappings2 = + ResourceRequestPreMappings.createFrom( + true, + new HashMap<>() { + { + put( + ResourceProfile.UNKNOWN, + new HashMap<>() { + { + put(smallFineGrainedResourceProfile, 2); + } + }); + } + }, + new HashMap<>() { + { + put(smallFineGrainedResourceProfile, 1); + put(bigGrainedResourceProfile, 2); + } + }); + // Testing decrease resource in flexible resources successfully + mappings2.decrease(ResourceProfile.UNKNOWN, bigGrainedResourceProfile); + assertThat( + mappings2.getAvailableResourceCntOfRemainingFlexibleMapping( + bigGrainedResourceProfile)) + .isOne(); + assertThat( + mappings2.getAvailableResourceCntOfBasePreMappings( + ResourceProfile.UNKNOWN, smallFineGrainedResourceProfile)) + .isOne(); + assertThat( + mappings2.getAvailableResourceCntOfRemainingFlexibleMapping( + smallFineGrainedResourceProfile)) + .isEqualTo(2); + + // Testing decrease resource in flexible resources failed + mappings2.decrease(ResourceProfile.UNKNOWN, smallFineGrainedResourceProfile); + assertThatThrownBy( + () -> + mappings2.decrease( + ResourceProfile.UNKNOWN, smallFineGrainedResourceProfile)) + .isInstanceOf(IllegalStateException.class); + } + + private List newPendingRequests(ResourceProfile... requiredProfiles) { + ArrayList pendingRequests = new ArrayList<>(); + if (requiredProfiles == null || requiredProfiles.length == 0) { + return pendingRequests; + } + for (ResourceProfile requiredProfile : requiredProfiles) { + pendingRequests.add( + PendingRequest.createNormalRequest( + new SlotRequestId(), + Preconditions.checkNotNull(requiredProfile), + DefaultLoadingWeight.EMPTY, + Collections.emptyList())); + } + return pendingRequests; + } + + private List newTestingSlots(ResourceProfile... slotProfiles) { + ArrayList slots = new ArrayList<>(); + if (slotProfiles == null || slotProfiles.length == 0) { + return slots; + } + for (ResourceProfile slotProfile : slotProfiles) { + slots.add(new TestingSlot(Preconditions.checkNotNull(slotProfile))); + } + return slots; + } + + private void assertNotMatchable(ResourceRequestPreMappings preMappings) { + assertThat(preMappings.isMatchingFulfilled()).isFalse(); + assertThat(preMappings.getBaseRequiredResourcePreMappings()).isEmpty(); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/TasksBalancedRequestSlotMatchingStrategyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/TasksBalancedRequestSlotMatchingStrategyTest.java new file mode 100644 index 0000000000000..706c3c8fefe42 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/TasksBalancedRequestSlotMatchingStrategyTest.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmaster.slotpool; + +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.jobmaster.SlotRequestId; +import org.apache.flink.runtime.scheduler.TestingPhysicalSlot; +import org.apache.flink.runtime.scheduler.loading.DefaultLoadingWeight; +import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; + +import static org.apache.flink.runtime.jobmaster.slotpool.PreferredAllocationRequestSlotMatchingStrategyTest.createSlot; +import static org.assertj.core.api.Assertions.assertThat; + +/** Testing for {@link TasksBalancedRequestSlotMatchingStrategy}. */ +class TasksBalancedRequestSlotMatchingStrategyTest { + + private static final ResourceProfile smallFineGrainedProfile = + ResourceProfile.newBuilder().setCpuCores(1d).build(); + private static final ResourceProfile bigFineGrainedProfile = + ResourceProfile.newBuilder().setCpuCores(2d).build(); + + private static final TaskManagerLocation tmLocation1 = new LocalTaskManagerLocation(); + private static final TaskManagerLocation tmLocation2 = new LocalTaskManagerLocation(); + + @Test + void testMatchRequestsAndSlotsRiskOfFineGrainedResourcesMatchedToUnknownProfile() { + // The case is aiming to check when the numbers of requests and resources are equals but + // having the risk of matching resources that would be matched with fine-grained request + // with ResourceProfile>UNKNOWN. + final Collection pendingRequests = + Arrays.asList( + createRequest(ResourceProfile.UNKNOWN, 100), + createRequest(bigFineGrainedProfile, 1)); + List slots = + Arrays.asList( + createSlot(bigFineGrainedProfile, new AllocationID(), tmLocation1), + createSlot(smallFineGrainedProfile, new AllocationID(), tmLocation2)); + final Collection requestSlotMatches = + TasksBalancedRequestSlotMatchingStrategy.INSTANCE.matchRequestsAndSlots( + slots, + pendingRequests, + new HashMap<>() { + { + put(tmLocation1.getResourceID(), DefaultLoadingWeight.EMPTY); + put(tmLocation2.getResourceID(), new DefaultLoadingWeight(9)); + } + }); + assertThat(requestSlotMatches).hasSize(2); + } + + @Test + void testMatchRequestsAndSlotsMissingFineGrainedResources() { + + PendingRequest requestWithBigProfile = createRequest(bigFineGrainedProfile, 6); + PendingRequest requestWithUnknownProfile = createRequest(ResourceProfile.UNKNOWN, 6); + PendingRequest requestWithSmallProfile = createRequest(smallFineGrainedProfile, 6); + + final Collection pendingRequests = + Arrays.asList( + requestWithSmallProfile, requestWithUnknownProfile, requestWithBigProfile); + List slots = + Arrays.asList( + createSlot( + bigFineGrainedProfile, + new AllocationID(), + new LocalTaskManagerLocation()), + createSlot( + bigFineGrainedProfile, + new AllocationID(), + new LocalTaskManagerLocation()), + createSlot( + bigFineGrainedProfile, + new AllocationID(), + new LocalTaskManagerLocation())); + final Collection requestSlotMatches = + TasksBalancedRequestSlotMatchingStrategy.INSTANCE.matchRequestsAndSlots( + slots, pendingRequests, new HashMap<>()); + assertThat(requestSlotMatches).isEmpty(); + } + + private static PendingRequest createRequest(ResourceProfile requestProfile, float loading) { + return PendingRequest.createNormalRequest( + new SlotRequestId(), + requestProfile, + new DefaultLoadingWeight(loading), + Collections.emptyList()); + } +} From 71fb41984cf8af69828dcd6fc70c1ce7d023d281 Mon Sep 17 00:00:00 2001 From: Xuyang Date: Fri, 14 Nov 2025 13:32:13 +0800 Subject: [PATCH 08/31] [FLINK-38611][doc] Add doc for delta join (#27225) * [FLINK-38611][doc] Add doc for delta join (cherry picked from commit bcd8d7f5d743b5344be354fd88b18f1e3d5a5f39) * [FLINK-38625][doc] Fix broken anchor links about the table options in Performance Tuning page (cherry picked from commit 1029df0f6bc85b892abdd0453101d037b4a6c9b4) --- docs/content.zh/docs/dev/table/config.md | 10 +++ docs/content.zh/docs/dev/table/tuning.md | 57 ++++++++++++++++++ docs/content/docs/dev/table/tuning.md | 54 +++++++++++++++++ .../static/fig/table-streaming/delta_join.png | Bin 0 -> 89632 bytes 4 files changed, 121 insertions(+) create mode 100644 docs/static/fig/table-streaming/delta_join.png diff --git a/docs/content.zh/docs/dev/table/config.md b/docs/content.zh/docs/dev/table/config.md index 1748fde1b723c..258993043e1e5 100644 --- a/docs/content.zh/docs/dev/table/config.md +++ b/docs/content.zh/docs/dev/table/config.md @@ -116,30 +116,40 @@ Flink SQL> SET 'table.exec.mini-batch.size' = '5000'; {{< /tab >}} {{< /tabs >}} + + ### 执行配置 以下选项可用于优化查询执行的性能。 {{< generated/execution_config_configuration >}} + + ### 优化器配置 以下配置可以用于调整查询优化器的行为以获得更好的执行计划。 {{< generated/optimizer_config_configuration >}} + + ### Planner 配置 以下配置可以用于调整 planner 的行为。 {{< generated/table_config_configuration >}} + + ### Materialized Table 配置 以下配置可以用于调整 Materialized Table 的行为。 {{< generated/materialized_table_config_configuration >}} + + ### SQL Client 配置 以下配置可以用于调整 sql client 的行为。 diff --git a/docs/content.zh/docs/dev/table/tuning.md b/docs/content.zh/docs/dev/table/tuning.md index 7166f44957280..e8ff6d8d2c972 100644 --- a/docs/content.zh/docs/dev/table/tuning.md +++ b/docs/content.zh/docs/dev/table/tuning.md @@ -289,3 +289,60 @@ ON a.id = b.id 默认情况下,对于 regular join 算子来说,mini-batch 优化是被禁用的。开启这项优化,需要设置选项 `table.exec.mini-batch.enabled`、`table.exec.mini-batch.allow-latency` 和 `table.exec.mini-batch.size`。更多详细信息请参见[配置]({{< ref "docs/dev/table/config" >}}#execution-options)页面。 {{< top >}} + +## Delta Joins + +在流作业中,regular join 会维护来自两个输入的所有历史数据,以确保结果的准确性。随着时间的推移,这会导致状态不断增长,从而增加资源的使用,并影响作业的稳定性。 + +为了应对这些挑战,Flink 引入了 delta join 算子。其核心思想是基于双向 lookup join 来替代 regular join 所维护的大状态,直接重用源表中的数据。与传统的 regular join 相比,delta join 显著减少了状态大小,提高了作业的稳定性,并降低了总体的资源消耗。 + +该功能默认启用。当满足以下所有条件时, regular join 将自动优化为 delta join。 + +1. 作业拓扑结构满足优化条件。具体可以查看[支持的功能和限制]({{< ref "docs/dev/table/tuning" >}}#supported-features-and-limitations)。 +2. 源表所在的外部存储系统提供了可供 delta join 快速查询的索引信息。目前 [Apache Fluss(Incubating)](https://fluss.apache.org/blog/fluss-open-source/) 已支持在 Flink 中提供表级别的索引信息,其上的表可作为 delta join 的源表。具体可参考 [Fluss 文档](https://fluss.apache.org/docs/0.8/engine-flink/delta-joins/#flink-version-support)。 + +### 工作原理 + +在 Flink 中,regular join 将来自两个输入端的所有输入数据存储在状态中,以确保当对侧的数据到达时,能够正确地匹配对应的记录。 + +相比之下,delta join 利用了外部存储系统的索引功能,并不执行状态查找,而是直接对外部存储发出高效的、基于索引的查询,以获取匹配的记录。该方法消除了 Flink 状态与外部系统之间冗余的数据存储。 + +{{< img src="/fig/table-streaming/delta_join.png" width="70%" height="70%" >}} + +### 关键参数 + +Delta join 优化默认启用。您可以通过设置以下配置手动禁用此功能: + +```sql +SET 'table.optimizer.delta-join.strategy' = 'NONE'; +``` + +详细信息请参见[配置]({{< ref "docs/dev/table/config" >}}#optimizer-options)页面。 + +您还可以配置以下参数来调整优化 delta join 的性能。 + +- `table.exec.delta-join.cache-enabled` +- `table.exec.delta-join.left.cache-size` +- `table.exec.delta-join.right.cache-size` + +详细信息请参见[配置]({{< ref "docs/dev/table/config" >}}#execution-options)页面。 + + + +### 支持的功能和限制 + +目前 delta join 仍在持续演进中,当前版本已支持的功能如下: + +1. 支持 **INSERT-only** 的表作为源表。 +2. 支持不带 **DELETE 操作**的 **CDC** 表作为源表。 +3. 支持源表和 delta join 间包含 **project** 和 **filter** 算子。 +4. Delta join 算子内支持**缓存**。 + +然而,delta join 也存在几个**限制**,包含以下任何条件的作业无法优化为 delta join。 + +1. 表的**索引键**必须包含在 join 的**等值条件**中 +2. 目前仅支持 **INNER JOIN**。 +3. **下游节点**必须能够处理**冗余变更**。例如以 **UPSERT 模式**运行、不带 `upsertMaterialize` 的 sink 节点。 +4. 当消费 **CDC 流**时,**join key** 必须是**主键**的一部分。 +5. 当消费 **CDC 流**时,所有 **filter** 必须应用于 **upsert key** 上。 +6. 所有 project 和 filter 都不能包含**非确定性函数**。 diff --git a/docs/content/docs/dev/table/tuning.md b/docs/content/docs/dev/table/tuning.md index ae372f082966d..81aaa020944b4 100644 --- a/docs/content/docs/dev/table/tuning.md +++ b/docs/content/docs/dev/table/tuning.md @@ -368,3 +368,57 @@ FROM TenantKafka t LEFT JOIN InventoryKafka i ON t.tenant_id = i.tenant_id AND ...; ``` +## Delta Joins + +In streaming jobs, regular joins keep all historical data from both inputs to ensure accuracy. Over time, this causes the state to grow continuously, increasing resource usage and impacting stability. + +To mitigate these challenges, Flink introduces the delta join operator. The key idea is to replace the large state maintained by regular joins with a bidirectional lookup-based join that directly reuses data from the source tables. Compared to traditional regular joins, delta joins substantially reduce state size, enhances job stability, and lowers overall resource consumption. + +This feature is enabled by default. A regular join will be automatically optimized into a delta join when all the following conditions are met: + +1. The SQL pattern satisfies the optimization criteria. For details, please refer to [Supported Features and Limitations]({{< ref "docs/dev/table/tuning" >}}#supported-features-and-limitations) +2. The external storage system of the source table provides index information for fast querying for delta joins. Currently, [Apache Fluss(Incubating)](https://fluss.apache.org/blog/fluss-open-source/) has provided index information at the table level for Flink, allowing such tables to be used as source tables for delta joins. Please refer to the [Fluss documentation](https://fluss.apache.org/docs/0.8/engine-flink/delta-joins/#flink-version-support) for more details. + +### Working Principle + +In Flink, regular joins store all incoming records from both input sides in the state to ensure that corresponding records can be matched correctly when data arrives from the opposite side. + +In contrast, delta joins leverage the indexing capabilities of external storage systems. Instead of performing state lookups, delta joins issue efficient index-based queries directly against the external storage to retrieve matching records. This approach eliminates redundant data storage between the Flink state and the external system. + +{{< img src="/fig/table-streaming/delta_join.png" width="70%" height="70%" >}} + +### Important Configurations + +Delta join optimization is enabled by default. You can disable this feature manually by setting the following configuration: + +```sql +SET 'table.optimizer.delta-join.strategy' = 'NONE'; +``` + +Please see [Configuration]({{< ref "docs/dev/table/config" >}}#optimizer-options) page for more details. + +To fine-tune the performance of delta joins, you can also configure the following parameters: + +- `table.exec.delta-join.cache-enabled` +- `table.exec.delta-join.left.cache-size` +- `table.exec.delta-join.right.cache-size` + +Please see [Configuration]({{< ref "docs/dev/table/config" >}}#execution-options) page for more details. + +### Supported Features and Limitations + +Delta joins are continuously evolving, and supports the following features currently. + +1. Support for **INSERT-only** tables as source tables. +2. Support for **CDC** tables without **DELETE operations** as source tables. +3. Support for **projection** and **filter** operations between the source and the delta join. +4. Support for **caching** within the delta join operator. + +However, Delta Joins also have several **limitations**. Jobs containing any of the following conditions cannot be optimized into a delta join: + +1. The **index key** of the table must be included in the join’s **equivalence conditions**. +2. Only **INNER JOIN** is currently supported. +3. The **downstream operator** must be able to handle **duplicate changes**, such as a sink operating in **UPSERT mode** without `upsertMaterialize`. +4. When consuming a **CDC stream**, the **join key** must be part of the **primary key**. +5. When consuming a **CDC stream**, all **filters** must be applied on the **upsert key**. +6. **Non-deterministic functions** are not allowed in filters or projections. \ No newline at end of file diff --git a/docs/static/fig/table-streaming/delta_join.png b/docs/static/fig/table-streaming/delta_join.png new file mode 100644 index 0000000000000000000000000000000000000000..6645cef43647c1c4c262126bc7fce56bc4280478 GIT binary patch literal 89632 zcmeFYby!qg+c%6Nq9|}lN~;WA($WYLL(0&N(%m2+U?UAPbjQF*cMPc@F?4sRIoW3IN@8r}caRkBkZU?yOutN%Z-Z}C4x4_%K|F!$=e{bkNkFFZ~Uz+`Ir%P$Wt{NTS{ks*MxW6y)e>q}| z`ZAu{SN@;RsMcswqJ=Fqz5^P~DEe0`SMC23o@OF;s{U5N6H`@JC#R_Yj#)F2*O_j8 zxi|htADV6HfbYq~{*4PYs4({mAT<#+$%!jBh-KxjnV{(F#1X|Epg|8kUa;@fDH_n)x@E`{?| z{GVX^B6yI<^zwi7-tUiHBFUu$w?MN*|B9qi+MZfm&^PAS|B0KcIQ*|8g$EiXl8LsP zSIG~@k?>!)mEzlIk~j39@rG0SP)S+2AV1$jR5>7%m51kYtA1OjQ19u-N*#Y%I=XqE zmDJv3;qj4?8DHCm_PT)eP4<#i7_E!fN+QSRJd@23$ zyQfD*^e|j_E+c#_pG1GY^hN7(U#g*wj@Nn)8jY4X-Kfwq;Hcxq`O5y!K>fTEQz6uD zSWodaptGxsn%}Bty`;HsWsmELI3weG@y)7n6MH$tOs(sJ#L>*&!MJHn$SJz6E`m8-;NXCQmI(AxCDnh0K!#^D#v)nRS<>~_4aZf& zYyx4&shHGMIx@0KhskohI~25HUWIjaQ-7jXS62yXgkJCU3QyNKAMNb;EGJrrslu`c zU-|e9mbtI=8P|I`y1A7b6y=_~y1HtWBM^w^tQPJ=W%gr*!QtWBd0OpD!a~rY*_fD^ zR59V6{!p9I6PwT!E zpR=6~zrzivy2huUtwF=4eUP@7yq5H$9;;CgWp5JEDt1KN=l#|4@s1l>oR@d!cqP60 zsvYBc`D$qSN&4m4%3FN;TSP=ci5nf?A<(^xll77gdcQEGb^A9$CjJMbZ{FYtNnz(H zOkzc}_f(Y2LMi&beXdrgl@2Bm>UZ`M-nejbTd`o6t2I z>Rk?aI|jj0A=G9-tN*KJJ|HIW7JB@nh5$cT!HIyvzz=Ygw~3wFiD#(ExZOe!$= zSAx+^fzpe=idy^)>U_32M#Fn!;wdxqBcjE&%Ai~fuHqZV$756*HRC*6=NXRD66A%N zn$8E4vP95JOnv+ImOUyaW`9)IRDT-D>Z*+|IWkopd`{ps zSG;o|&kXGn_1>DfI6u|I`1trBODK6vARM`FPY@9GfImNj0>RJJlmQVoy6Qzdy zet8slso(%b7Mngtt|<%>9bx`qgdI{*odLY6M*xwqt(+TpFVxM zAJxC2o-7nj-*mj3+!{p8!lnv^+KNjKPe3q7okG|sCgr%&*_XDF>FM)GD1uofL7Ii& z&YfgY&$Z^ub7x{=;%Y}t?)VbDvMdnT&F6>7GMQ$o@V-Ag9SjH(cWS7J;Y)ae0=97QIQK%^96l{|$0KOzr2>qY z5t}n;_leRM_T$A%@@ly`Ij##3klH%86dM)y=IZ+fGG)cQw^Ul7)bc~C%ggflV?~BF zV%rUSLpjQ-CkbUxf*bD`0?(o=_rN!eKDD&8=vP=t>~=C*_osQfy84`8QDAx`BKD3v zpH}(*c#D6P&`eDFk|3#Nen(r*&F5YY4xHnO)lAGHE>NH3=D^|U{Y4w6q zQkUSb+hB?&E}o8qliI~-!mmfD7+wYiAsqdeYy97?4;=o}?Vx~Wc=C2bz__XM9ZowpyRKQzFtbG~n)qUwy8fc2<-BBZ;Lp%0 zIKMdEK0Vr2N39N{`awohQc|MU#Y{l*deQ?gd}uT}M%CzdG*@9wYRKQuP_Z)V0wX^8!w-*j@ zkBw@_X;zjzGZT}vxs2?NNCpN$yAgbb*HI}cw4|haV}{Oyt=S5ZsfT$v=;;=1B7zjsV_2xT20Fo}&uDn^J}zK4|c>-F6!1{f$ZYU*Chf(pf{YDcFE4nrxUzz9KLi+HDccrs(U!PAA0FXNlZN0^_&ccm(DlB;4VX zp#9j4|IsW6XNyyJ_@BnTJ~8hZSN!9})xm6H+9%R!^0^&JNcrrC?u=5%yN!3FNV)aP zk%fg0r-z%!K!~55oLmf|>vAVjgSa{ zUBp~}eb8dJ+|$+7)%*#0wa=Lv-Qaz3c5G{F>kw~HeaazrN+h<;VY}B#EcJT1ClM45 zQ&Uqub?W4>?d$`9?@Xd(>{`A+o7|>nA-46BYw{TqO{{5`dLfZQJ zY=7OFko-dMF)J&prM4qag*p4Kr>EzJESwkC1@ccyN=i}^b*z%Mp`mh6g@p3K%|u8C z2Nzd(WTd*fI!kZ1d7cYwc;*HW>D_+<8emxI>48P|2RJw!{|CrQ zPiJVc^O!a@yxMu$Q-b>@{NHI)C+lMG)hCcPrGun1pO?O8DKfr?gM<4|W5$gp5N!Y- z!7cuqPJxd&X7vBu^~lso3M}}4Aou(a`FLW8|D-r^@6m<;|9hyI!GC-jF$}5V0O;QF zZ`}Pi;k#&m>G?tLI*uqO33@rkyX^loIZ!FRM*WbCAoMPTg(Y3Yd#f|?Q{`#NLg=`Y z0oswJW@UzHrL?qkVqyZo8CE8ouT7cTLTH%%e*yLKg&{Y;ePW`?rtPoZ$wvVV2Y-SL zBbyt}TVy-T73l00k5W~dTFYMjm^5cPQCaa}`fxCl5$8sJ`k`=+AfsD_rIro(*|8`earON$C*jtMf<8>Ri`9Hz#Fzi}Y!fxESv~H&;3rmn1CeRU{)9 zr~9qRGJb>>;P{sLyidwxLp^a)A`<`Xv7`mI?4?wuf!CLT)9_Bg{V3HVpML$=CB@@Q zo&aH88R449xzBp4AA`m$8pzSr%_}|MOiZKPsRa)1TPGmnKk4+^**B-+ycxT}Y`ao9 z^KjlE@Xtl-U?Z)va~WQ_97b;90Om=|GB=Cj(@VEd&@eciz5e49r%A(~2(c{}S62=8 zSm9ok_)$Pl_5mC}gq3h->`v=PiRv$0kjfaFc=*xv zF1EdsTk?FzRo6+n4-arH-#w0W)X-WJPzZh995#RcdE48F@x?1gisQxaQI?CgDYOy9 z7lQgm-<2K|1UmeJ{9dUv54z(|+buJxl|z!}k>r+>p{UW8LmjK9S%R=5%kiw*pf^d) zfU(8cu6vyLPW9`CU>_!Tt=T=JM(3Is7QsssTkZJsEObq@io@bLmNwaHcQw}rvAl+z zB2YeEi?p1Un&rB?O}lv5oJ?)^L*<9f2<;(dE2iNo=Z5nPSf1C$#Lk@GHaQ@wk1xNy z$Daqx6z~mh!|Dw$UE&Fj$I$vEdDZBo^?`WBfPjm%1(fM*z=cyTrM(a3O42k(l&YXc zx#aOw#GQ5TZOdtvqVv9M12?P^Ep&9o00}b*JU}vNoZSNOG`^XQb+}rNkwB_Fh>0^(mJj&uZ5Tf-2qUc+y z!v_pZY@S~Cp08maW%O7iyfACkg4ZW60b3JK~t(Cn7!YxV&}2!W-o$9n5OiF)T|4qQqOouH;^403+TcgPb#S8FE~b(QCR- z+}qdCpwX9(pzd?i*>3k+T>SR7wpjuG*&DvTzOdy?*&ww22^6_{AhlA zuCczfG^&_yYI}3D(E~dK7>7GRnI0qHnm`U{c9`UI>jr$&0|RF=ta0k?=^3$89S{GK zpDzlATV3jWA7G7?C$3q!xu1%M5iIH;BbEU$4$AZKCqf!-<2uM+w^k>$)3<)pfRl9wf4Pzt7SEBoQE@&Ute}jv8(Cw!jJl1EAlso35 zWp@`^UWQMe{rfW_=OC&%qP0qP9Uj7>q$-Vf8!AxS;;S0a}GFBnYtG=Jn$Gn|JU39??|)`03M-_TgsD{7_mS&~w(u z!e6_Me9-_ndksL;ae%(P)B&O6F=FwC{EO6h#$;jM6eRT#trJwbpzl@9gWVs>zDzHM9&IpE@O!FoUvz zbk}Xtr+Fcz5~|m#8=TbNE+o>YI@9lyO~aF90k8H$2hJSV@v0FSsu|>rXlC)zhA-Op_vB2?QGKfG!S_z6nOH)?B9QwO4=;|I zFPWH_a!@EhtYSv1*VgRlBNZlBC)qS}YwPP*78l9L$$>Tqf25Wse%uwyK}AJ{^kQy~ zO#ynuH++UOpmLIslCoW*-B-W}u}-Y4tOV|~Q7)#}du?W+K(DXL%+)y0=W7?p$jGo- zJ2-3rQG@k;P*6wsE<(O(w~M1+>hcUIN_mp|zdr)T4734&SRHa|hKs*Y zR}b!3U9@}qa$7zA#-zP0lQA98n}EQ{T#?0d22>7iquSqJ-{N`{Sfzyj`k0-)5JJwe zItk);?#rH?IRY6P@n|HD%aFCa5PbK+gEta@@32WAqeg(>3xvXtU66F~z>BkS)4JbPI-u}DUC0a(gvlylSZr*~%h`c$(fNwkV3?*; zGWWLQ6Y(>}hD`xEp?kZjc<23#Dsyf`j4W?A%^cwSQpYpy{e;v4tdf#x#e%+i8X8aO zCxD;`6YEm>b_%MbKM+b3`6dI+;B`P%X*xfg_NX6NY~BZTQz7g=FVJ#+{d(D$xqff@ zJeKcre^?#&=G~0dZqLFTnmYuI8y`et9FKRGWIx?K20}CvX1j1!8AKM(2jo!(zQi#4 zp)vqOvC}o%?cp@8{jI4s~>NHWC71tE&If6hrkU6@?wa8 zO!%XFZB;K?KMhVQgj37?jO@ycAATaWNqOrAnE*C+C=bi&vBBRj-b#PnfhqwhVMfxR zn)FCNya$7s4BY=d;%JSZ(p700zGzY2f82tiYZ%*@AWXbb{iY1Md;B^i0?0HM031s4 zOYbDN(0`O4tK39LwwnNK@+genjA?KFkkU&+{-cX(5(S#&)Hr-1{Z6@ToI_LMg7F(NAtx24}5xP2sWhQ?d>fZjd@FUuqrLu zy^kGkY-j*Uk=4K-7$N(7K$OoHzEMKTqBbDg^>qn#4m^pC!R!toF=EU#H8p|iEJgq$ zE1g;!#-X^5a5xf}CxZ^8KALTC68E&7;U2$0RX7$atcSxxOHY|DJ}(4I(LTXXyrI)E z0h!FxKi$XjJ5E&|74HFo>v<^;2gfi?<$w_a`J9@HN(A^S0Gz=pbMLEr-*notEvbAceOSFw17S+u8f|LL-*Ziwr3})Pq zF;=wjM;e}<2cJq)PI)K|3+Q&`^U{-l1jNI&U-QAF#(YKqIGYNtx{7U<%Pl1&ANtH|B)pd5jkud(Zi> z)N|td`&G0~LFZ~Oc;*96xiE{ls>v!!{6f%^-Fe{T0F)@~G^5zIMhH{FKU^EN2lC&3 zl-jeMP_<_dgBzQ(x}IBPD(-;cFC8z39+S=1d2OCdm`9d%GY*gTMjSftN*}gZtE+x* zYl{|zsaP=F#KRkG1?A1{7`qMTxGGN^-Pki?tcMnJ8=n`um|4YF$h5HV=AaTa6+K3eoOJ7|#-`xxsX<*UJt!O_juZ@Ntnu8Qe+QxdB_0#cv7 z{rZq7h?}asT+;vK3KILrIfPutx7PG9^*-FthkX}uUrvaza)4J#_#HZsGy^LtdNX7f z2$PcM`@>pOJk4h^iIXOI7I3m>wOzpKE0dY9aD)PxWv_Q@wq6PtIO}6Y9GsjE7t;Cf z>+4UKdy}zYrv3-yc?jqQ@a$!kf1#3yzvcy7%kn*6aI&m5Oo$#S^QWxD`6o>zEC zv08y+mT|~Zo%;CT;?5CWFxCs*wZ=Q$qPWd@-$m%C5!WeAG9U_S#--8@RZDoB2}}s) zZe_R2^l()^e*V2l>vaDu^a>!yNu24l@blxi;+MdD0!Ev|$vzfHd6m}Kp`1eDaz<+c z0s>H?gzZKEMPL;(i35_ti(Ud^y`G?89v4pBpoJ!6C2Bt%o2q?M zY9IXK1@U?i+zQ5{`%Si(iGET%e2ak+Pi!zESe{x~*S6wp{`I-U*ge1})dD zfJ0#eyx5Rmi;KvbP6(Ed#qaFO@<6Yo1)QIRQ}MSC4(`a~!!H^ER|6H<2Z+-wYAJt) zlzE~_0jP-euP%e;=H?EiHTj=5S%0sz9cCqZ`NDz*pLx;*9WC!BVE|C{Y}-O&E{!$>RF?=RgV+v4;C>93wuWSum_G}fO-M=0$4DA9nF_wt6T~P z2RI}ISOZz7?*S)^dE|QK^=t+KsXHL^ezRqF9B^x|JUTKIfb`zg>HWpV^`D3tE*25$ zFgoCEDEJ;v*_UWc5zihWvoO4PCH;t_R$ZcDBhO%eNF{cMU;yCZ|J2S*JAO$Cp*%#=^CLYij1I{3Ob( zuLtHa>~Yq%3t~uR3bN9hJYMg;4V;W554lX#es9tuFng2pDik4I7>s_EopyiP-|oUt z12geeDII+c{C7w?aBHDZXt?T7t$r1ts)yqH1G3te9|-fl#sR5DnL5aYJD@_sfGq@t zLX}9l?(E2Z+cajJ9BD&)O<*j-5^s#X3yXeKnVd{>ddg~Sxo93KzT5fc8gA#n>Neof z>fd5LMe0sEA<$_M|C6m)Q3;RL0bpdaRcczno*?G+p$eGuFRUb5U5|m7R#_=ft)Thz z={MjbjMq4u=HqOlShuiP5vQ5jE047gC~gC4qO;)hWJU5KiUzOF6PVVyCwJ+^6N(;V zR<2+~XJ==10K;Z9d#2sDzTD)b&DiDSN5`)(P6jqI)#Nti7)25rPG8;=b|}XL6t(xl z1Fh0X?a9V2Sfz1A{Tr1~nd;wDDSL%9%8gANCs~q(p}vaLUgO5zgbb46;tfBwV}Mhq zHza%vtU^^KekvC1j~~+93VvZ7RBkous!0~o(l~T2;Z_1~op>e#xP4``+G7j=|H1ird6>Sw+*B=mm{Hez6WJmeg=E(i zd1}nz=e)u~BXW}`6N%$Z0&oQ8TOWDgr)4K4cly=iN&csy9YLIy!)J@{rE=(VJjx+o z%Q$1;x^gKpK^&dc57J}owLz@wIWn&!)bYxa&_&-Mn-AAYk9zzN)hvpNXxO4+Uw!p5SMPp=LYU&|B9v~h8*~VfvW?Vu^hT{ z?)?%#6q+ZAX?C7(G76XiYdcK3WIas)*#u60=Le~i)$GgD=F9pbhHJH+X7J6hI2*`p zB)ypT&(U7gmUy!L!Pl82prlThS!|>gaMWzRQ+Zd7!UT%c-Njya5Jwv!64wbs_XYS# z>HIY37Rxr;02>n9I-M&w?|=_2J*ohWqpswiE^jQECmik$5_0V{S*a56rWZTR;!nBlsaXAo`f`hbFN`9zpWOf574r1ou!YfP?|yyISrfX2q2N>Qk>+7gYM6 zdU^fD>=zQvsIWCrwjk87lOYIF5lhncNSB?3s%p5m%POkAZ^j=|T;YCt3jM>3k$bh0 z*g#ohD!Sl7RnvllV<`P-JZ0C3f&cZ)Bh{M^bQrPRaa&^JH?5~G&P2JOjYfFeuP9e` z?C;r33HBJ3P~(Yp7dRez=Oot@7IJziir+C-stj_>9j4(Ht*c_34-F2!O-#(g%}sIZ zhVV%aybIIQvnZN8K(4g3xVQ+!?T;Tm4BV8k2<)<;6BHAhwir}o7Z(&11oNwl_m~Ur z2H$OKYdbzUfotn%Y2Ej(dqE5#R8v>C4lFFp&sTy%GbN6Xj{#RO{s`~pAqi$>iBF$} z@Pp8uF)NJvpoFPUZT?#Q z_31t_PO0d=1Vnf0vOJ~N2IxAdRat70iA3vz`-28pBI}ExD5Iu{O5%q$-YyjlZVN2) zVS46bd-Xz12(-o}CF#2T3tQj!H-vmP!o&9Ejm`Ikw>}LozHRq(+O-t&$+A+r=I~d7 zMg6zhr{&P_!lEKoB_#u|!>%t2w-58939rTieU@k#(ca!J!+Z_rhu5%*_x(f_`D>pA zcDF)5Zq{kqF9>v^caH|dqe%+a>*nH`u%_X!h8U zb;)K^>Dqt^2xMHa zQ0&ek!AF@#(YHRzc(q?{o)R^cPS5t8R#CIaGK}ksslTMx#d$+swv9Fv3AX^HS(Bq6 zn!&iYLO0xk6tg%zo-pbC0CD&s*CZaoo>y(rNbyT+yb7(nksPJ|WS;Da4H2*+X(Me$q*m7gWxNgrqxg;++YZ2}oR5p2f_v!5uLn%(7s4R@O^x=_G%Z zv8mT(2?{4sQ1jJvlGj@ROL{5FLbspk${ZbUpr`Ls zgFF6)UBG%A;Jf`s|0~2XiAGS-t(B33gF{(aSyondg`r_!kQy>+F~fQL3IuJGRi>C4 z#UNs%jiTD`o69ZxXgoy!@)^4;xU?d$WX)>Zt-ju+u93eYn7U7nI*1KKorEy&mNJzF z6ChuP_g1h^KS)G7F^N2#w0(-_!&Ye6MeaEkkiJ1t)jrqsnwmJ$0)M4~8al`#EbJW; zKT8zpz}L8?t@SA8;lKpLMkjQrlR+~gZ1=*-b;@;GpYa9`0MWL?0wRMEGMSIJ#*@<1 z1Am>hMNB-vFk4kxd}|p^9^|||MKAwA*-hV%jKY+hQY8Prt+=vX2RbX5YHsV<^5lcW zhT$nUd7hlMmHRb4&9DSIm{tN_&JC_ijK0^R8~WL!7b~w$houf8x?{(PuQ?`cxh!9P zl6$&AsoKbmWZuNQVL0`Dy=(VJsS|?3b&c7!ULeh;18IFPvV()2oq&+g*kvNmTu8Az zuU@n5gJXP3YUoLX!HcgJg}J%>QkA{$szn$%H;kL7cBfh zZoWK2Jf`fPJT}0){#09A8;GpH<=kSCe4{E<(3PN=C zP}PU18%~rnDaF~Zn?ww$9X8iyaOy@M<{=al!lw-U(d`M&>4UY>8dvbpp*1AU*( z!%1@(Tv8y$Rk^$)mY#s%oo{B{!=XUn%~O5V;KaC81mmUCEJIio0FuM}B}|ZkZU-GQQH*K8gCv-_Gv6HZx>3G?ZrXF;AqcvXB&5oK%@Ev{rls%_NKs znan`dI#wCVft$jG*ceevRC-@_{TLq%l_G=uRPqEbVJ z8p4)BBdoZ4dg2!zW_maJm%rOjKp1ho&?TSt6$I zMg;G>_~>u7w2p`$ZS89aM4~l@ z1Ukpgk?80qo>%#l%r;p_U_9tRXwK$lSF0uem>PRZQRMT90Rknr;C4->8fPSj-lF zJ9Qgu2#}Bd4l*O1O2GF;Y6yA&5f^tDrP*}QyjK(tY~lH4o#)IlD=uO|7JP+53&$C4 zktbZ0Nh)?#VWm09RiB1Wj^zq&;&uHXPK_^F1Spz?-b*m97Zf^nErn^9Pv_-@Y0EH+ z;XH;KB6rwLmAu<=NmIB$MKbd{5az)8>c{g*KHcYe`@ z{PNOj=ZVkd%vUx=Vp2>KYbls-tJH!{oGz&{Ij}#5?N}!#24JfJJkjogld5KF(znlOSs#`^8aSQp*dl1Qyv*lm#n8gCNu7Y? za?8F6C0J=EgwM^*?H#7puPb)YLwqE_Qj&zqX(k~daAgBJ^KdOzGO!{Q{E|XZcc_tRl1Hhid!XaR$qe!?8 zJ}Rfw(qLY7yuvX=^my)m1sl(6OX$e%Q0>9!ZRH!cr4(TSTF8qpeF>)Nf{{i1DkK8A zm;puv=7Y>bFfpB+w$ydSJjQi|JiW*RRy99ZLF*MXOF~JPo9>}=OFCHGNe(Lyuvj(u zM=#nr;418~j{b*+AJ+Ahm8hs}H>ciA-%=S`wwZZ=DSButF2g4xa@;VS<9%eb=2tE2 zE$|B#-#uy`@hqi&vFE(dX;YnK;`WS^_(rTEwZ6W7;{5xbin} zR@-}0haFuLrtAv=%sw2E=M@6JJ^>@ta24*@c4toY8LhNR1mrg}o9&$?&d+mS=?qD# z-~@Fk@KqZ_~5;f15~9c*#IGy!5GH?bbQX`$vl3hD+}&&d0YMGJdJ6>#!LH6uvj)f`dt zw|tJ0x9K?&9Z0?h>e46#Qr;qIV!7P_4KY;|VZsD5I1R|P-`B>@53yAy?j`NFx8G)- z44LQ%6APz-kQ3ldKLFi}xp;>jV$2HC{CLp3>Ge}7ig~4FgadCASo?NU-1{UCJO+y=bVgkyjWSO3Fv1`}|0yoU07T8RbvmhR5$a7Qg}+BrtznA~zGd6@}D{@O8f@5xvtgq%=Ud zR+T&IMrao#CVIvwG0VyGalBq7M9eHMO{QpvwDsIA-dHT&@DSS&vPC|(nazJYO`tMV zZ4*$O+SS#SmG!7DR8>*Y#ohhXs~`JQVWEirxg*O#o!;Tj*up+V=6ensIXwiDBbr-Q zGF`o91jryin}Q76AVsle@FK+`y~d4_!$zdi0-GWFCe%q$PD_idu9_Nkn_f=tIvYjY z#;qv%G#*0#mdjHrUc>3FLW?jonlRKZ3T9;`GI)OOCes% z3C+2su)&>TX;C)J!}`_Fir)*BTDY#sRK3iJ=8K6F3PXg2%WdnN-ek5l6d+nL8Xp5| zM!kwt!9weVg!?QkC?2oWb|rXwaIyzJozYYKZ^n@xOd-!6(MQIEEid4;?2p5ITTfu> zBufeGsB%>n;J37wlf$F&27|?GU*P)7%0;fQ%unE;E@N$EFkmnJj=_vqHWTK)tn2|V zZ;#M<Z}eH(_x;IMHLJc>qeI~hUUU1i|n z$B6>k^4(8Lg=7Lh9DjwTrl+dF` z2|ZO+C#9t~A0jt@yU0JC#I;Qpdkpz7nR`w}MaT@D_Mp&tim}D&d*u<2(bMqQPp%bZ ze{BaV2>)WX_YpLahi^_%GcYiakhuMs*b%NyZ~b1>p4YvUq|x#6Qq=z{^WOm;Uw3J3 z?Gr)4SnjVs>`!BDA49)BM-AaIO-&>)x!brE(|JV*dnw9+HNHkhGcrd{3IZ|HT7XQZ z1PG#NWtqfo%2YY%Hg&w=NIy?-LvHp>A(I6Y!_T{m8lTcEI&V}zYxx5fwr3!7Ey4h6 z9};5uP^n6w#7d@ea{)K}j>XTuA(Glj?>DK}5tR>}D%k~u?K$K;s_pHirN6#qR6uPt zOq@0_`G@<~h0-4&nrN8PsW<}pWxSeekfGD_{a)^*vnyO_B+RF#g};gvCrK~hV&ZM` zWtsx3A*)iELo4r){^Hb!-k#*M?y3SJTB1VFr)E_^Ua(Jm_^!yXX2Qw=cW&iIaf);m*UM`UVJBpmDe=$u;_! zM;ixeh+Mn)jMRBuflZ4go3iNm4tQf+_~acAiyGmb8Gd+Nhd zIn?>NWcnn<&XWGbSJq|Nsn$J9<{7r^_SlixY`Q*L`+&_p*&_2YJPHN%C?;Q9xlRKq zO-HR3+>on#=G~+sahCMKDEe8+pYjLf^5nxk1tf+8oU>bNcR^C(TO7p>t{Qc#y^wt@l? z*hV}nyG{1q=qMH!ce?SDr4`@p2_DcnE#D#N11ZqWU>Ulw9uv?M)`Rz@W&+@zI*2CX7e6ORQKpYFkIi7P3MdMfWpwM>b_G<7(GnHjf>Q5bm~MU zZ)@THWZ@0D#CKgI-6IDd-HN2!QaEbm`KQ!RqDmGW@bgUMoi46L>T6G25K425FJowiN!6^ld0j1L$F z=?&i-7+Ih4fI?FvsBqaRYG$dN@fTY+q?OE2AftU=#)l4Zan@fbC@OXWsd#LA!tpKi z@Zc}UE)95t-0e!!@7?LY;~Pkr&shx)6QmYCY%5^k&-Rg$wFc*MDX)#9_8VEz_Gp8g zf)$SSKt+xH-N*fqU_0&exp#HrEtVxg%BWAut#$%h0iX25INq{d&MHnuzZ^xwMiCeA z5)v{JRsW*83$wSo$I@8$w?j;}1c~x2TI1VYnpMB?=CFE|`;ntmSnU4TH^{dluz^@u zj&qpS!@j^OWGpvP-w<6~bno81`qbimm;BO+TKXgFEKK}PhK$fyh*?Bl(FO&yn}BfAesm_K&WAH?6?A!%mdLS?n% zE7wz>40`aUoNp=~_dIE7|sG@6A*90K_nFRY()p&)6)|c9!_}o?$y4_dPfu<^JIr? zrASbsoTy=bay$g}K&dRQMF`E16PxJiS$FaX?ewBNtK@}>xw`itOBtarx<>jSqhy2= z5o3E`t*q7aL`BwodHo*jqmF4o{ye;(q4lp)zuI7V-%y5~*sw!#239k63dRBPd7SY$H~J*FJ1TdqIjLo2r@9ol~Db$w*4_*q{3r~8@tYA#J?*y zjQ?G<$OTtIb}gc&5s8JrTiTo|R=c z3B}+%2AkAN&1*AS3E{{~cRuJsI?uwdUjPf$7#)`NK;}c?x-?Dg6**J(v1@L4{35;h z8icqM)hlcqu{{-hs{XmpIJ9{CyQ)rpupi8t5HiuSYvrvUU|(*grUg>2n-&d@tG0GnvttKDQ`6km*49VsPF|vm(@{3gP5E6eUT>A+8AsVY4@6}-v$N^<;^psn>?2`@ z42%O1f~x%(nSy46Jo+D#L%LwwlS-aEWkSRoauq%yo%9(K>pi;+AZC0Gcnh5(P*uz5 zP^*zWDgl)U_yRz0($LUoYHGULD_Utj{(Y0u&xBK0xHxj3qNJygBZjy2mZgDzlaV)4 zQIK0tv`gW2mY$rTE4!}L&&!%>iB(2P+X?kSZ+IeeBHL)=y+N0b$3H@BG(uVyrMb^J zq-8qzwsHB%G9TI!x+Z4%Nyl3KAm8aVO%44`)R84uFgCmyVC1mzk>6UV(4=u=+*G*O zq|y1vDPB2#Xn2Th_ptHJ6B~1|=_Xf^ENOTSc-|kG1o#!<9Scf?_>2v zy~Q#ATQ~|A5=eE(>Ulz(O-;?^oCGwCK*By!R;|aF2x-G)$hiVE$2YN%tIzY)=BEsE z#wR^M!r*`|SBOSou@%X`RV zhWqAJRY__K=YXP;-yRi0Yi&)cRuZgO`IJ~^Ywjc8y2vzaPTf;fQD9?~fgH;S>{}w7 z)i-p4?;Eg^5H_@YK&NeGnGUb7d8m*s`9%+HjjzxLj;PB#4Y{r#{qB0i1uSu`cs)EF zWR#pwrKJz+i&1{crax*rX0l!`h&7fv}Y-h%BUwcGGfPsWt zOa4AQY;yV8@~LQ=*-_Gb(ifh3H&m3Uk^SPa0fk=gnT|)H-ob}-3wt8|W(w?l*hI7H zc9Eif;hW&%(>q$(uM=LKIYBB!xjz1!yiFi_WYf*w0^d4Ghlyrkj0ZYOtxNhoC*rz1xJWo}lxmruAx^qBkAM8+J)4F^=_o6sq-b zv-xtjdrj(M0`0k;XWsFswIx4)YRC6i1|is&jf;!>0{$lZyl>Vw`zhRE00VNmk`l;? zd;9w--XbmKdi)adeL`bPogNXEYl*HQV#UPX4QBU+zv-@pt)Md}$c$Rg`r^6AO4lab z5QT-qvtWtD2hom@5swGznG2#_ZniOz&tBRvvuTd_=?I0Pe)9?hk)T4g8x4gH)SB<) z?CCm9)drlumGEn5YzPSr`Aw5Urb;ECF`(nW18B;rm~C#Ob$DEJ~duOnZc~tg)hvemN%x zx2W=#7oShIxm6h%nwp?axuJutsnKAgD-Q98MM!G>7>6B-Det$2#@XFVj`OMyX&06o_ zDp|?0n}zIk{OO!DY)ts%(jp}&r0J7c?~z@9J-&3FkfylY+YPiLFbbXKXRAS~tPaPY{^LuB)_qZWiz?jbkMYtFaPq-JL**kQ$>+-G@|G$* zE;DqR*>xJCQ_@&trAzN|fcF?bmX(<~+tVru!rAv%7M9BP9(=Tnj04@>n?p{B3CYI3 z3KRRoO=P00Qv#ISZY#<8ZL6uIA6^>XP;!fAzvU08?cs@;)sAINI0!#?Q+?wui)}pB z?oXaVeJQcY#53oaFR;brSz97kwQKUXJ~$2Jh%2q(uo>TnN~56!ZU*@tG4arVPtLBj zB4Egt&U>Y*q@>7T9(Fz-3^h#4Z3SWqVahwGE4I)Zctr*08>pmYf9Yb~1J_>u~lWS1$of)^j`sN8mCuzdU286gMrzj}f4eIq+N z&P3!Y{U9dhnzPn|9mPm*z#tXUaUcZbU+ycffCpE8zN=y-LU-aS|9R;@>* zPq~GN-FsYp>1y1H2}E)#ISy6jxV@aAQjfJE2?}bj8H?gXk96lMjqBgpfwOv z#7O+~0)|c3LI=)5$lzI5zC)goSPD4R_Gk~Q@BF&@ncO1U3uAk z-eBUjJ1W{cGZ@9i)g*apZ2FPs<^GCJvyWbt@#POf!8&VxpDn2`Qc@=3s6L9%JWRg@ z2kE(fiDhMFm6esfI6ZA>vhmUB(k^~M18yh&E60FALA#4brm!R(*(tY_l9540PJVE5 z;tQPkt7AIAB)MRid}tjayy>6k{w(>Td~#ggb(^88XE#D(C+aF*g+xuop}Fl4%_{Vb z++&Cp)_Iu_B_kV*YdzmRb}7F9F!#c6VAV?T>YM3`i)>c1P_FIg0`4f1lnJvhT1IzsTw> zoW2fjks4FGX(g>OG5jq_Vjno04Cs&kvmANRsf58|gq%l3h~-b%9v)jgwZ$rJLO+`2o$9ntbr#=3 z$b?fiLBexNPtX5ajs4y8HU&Cvoh2dl-V-tSG7C@hd2*wccX5)MTvn5hV$lY>C}#Et zZlFlX!M=Z2d4ab3wO#vC-4MMVgGS^kiCNcptbWFIjBDM97xTI%14*&P(2KM*rVlGQG5=i+v9OCJHQbLTv{>P_xq0 z=OVjYIld|;k`IS2bjwZ8iB7l9SJEzH9mQ&Alm!`kC6LQCEZlmarle6vdDbQNW67Am zxpZOGtuR3I?yOyc@H2|E&d~MtPQ=9IM^S1$_EQffvV%Cla;K1InVZ|z_SRZPvuJ0p z>6M4E&nn^bGbE&qoiQBG-!$#E%in*B6kCtcYxsCbbbcqzjvyxt4NZEzMFaq9RthGX z%|xbC4|uOipc3NZmDPbyEhlj%^vUeAYK+O1nCiDFA(|1B>(&LYW1988l2`)KEYYkGYVAj3A*$BgW^XTXl zU%q6BqS9&fy3K5VYBT8$YR9H{etNzcpLkFrFu@`_`6j?}zRW8MQC;|+Yg+EzrSgF8wR%guvkr$wy;p}3uf{gd>b57o zPE@{jW1|vApe)kW3)x9E&aY`(z!0ZaqH#UREBPWJu4ZU@zVWIp7M_Ytkyk&86ub4x zte~)LbZF%A`VW7F*;rMcj5-UDnzy{i4WY zt9|_6N2pE{ip&1=id@ykz-<<0<4%KaZ4CS&VA`Sa&JULM1W2$JGrg#&&^l&NC$2}~&%aVJmcK52ik>i{(^%D@)4skk>P#+Z81XfkPL0R5 z!x&0=7@_+q)&Yh*3ZumgyKNa-vL%|kTARmb)qLFApx|YIuLFGE>KfbQ^zTk z@pdsKrAa-ImIfAfFvOZr0{~6XZGDd}E2>&yRq81qyCa0v7dvZHej*tURFjO=&0Q~xF+H?e!lH&YZ zm5mJa8ckRMnrHf;JfHn58m74`laS>E?|Xu}PXjnnTqOnnB9q3Vh6w{La) zg;tsbR%BVR)NVVUkIm=2-;q7n@Ojntk!f*52(b)!4b z&HY`nI?ffmN@>#F5UHM9+)R0A1v?b9_y%>PM(gdxpyFVAJ2NuxzW7t_-@$z~CPGt$Vn+g>?ONskTh3eS&-WPjXFOj++PZgOQio@Pqssd`K`-j2Cu>0ik|Skic_ z@8^8@rL)NeBh?zkocLEU6!3q>Z3LQ+36=*}9m+bs+wH8+^*o`+jE2Yh#unW0s0s6{ zc(h-y<`7E#qB?!qPCyIfP+ktjKeoBnUKP#(^%lF6f7QjFuwn91F(dNP?PSc}Jql+H znOh_FdsE-Ay7(6o-ET7Wr{aT~n|?5Ir(X=Y@l)Q*2RK{*7oZgsV0>1H0B6aXu6)sTzkRCyO0*`Yb3%>NhDBoC zK_W^YPHNQEmxPS(vq&Ov-~Ft+_Va(o;7^&wKKMJua$j1D`L_LkX~ z`RTE-aov6QURqwpp?d$^cNv5%ip$$DK3EwWPp?F(ikiHJCS8SuhI)foMg_UR7%=#v z#!~+l@%ZW&N$qn(<7@{AMB`Y$5OuWyE{|SVmR`4crk%-#>8K|Q(o5RZ^SbLCB1wwL z8-(JA$)T_vHv7aSb@X8V_O&m zs{4`wTE%{(s;8$=!hjChTVF?f_ZftPpJG)!C$k05YfJ6RMRIMaDJiie(&HhSXP)0; zJJdf*=I7`A6qsLRk&=-K2`OM^Bq1lcxxFFS*}WF=N|Lse55s@GzKY9PHM^2gX{R@K zcj+BQT(8J@9u7$+z~EVfuZ*OUs+ZN!L!w9FOa-L}Z4Kgx^`3uvJZND&JUkBrt!{(Y zyJp^k>i0&s1e>>a37dkf>A35kXOSJA{Yt*Hwr;Pg6X)aUsx->Tyg5jkC&I7SsY$_; z5Uk>hb+<>ScJ$QJ0$2TFUy@?l+5)1ZSu0(ol$83N%PE147rEmKw3-N{L4koxjEocl z&WtQ9FJiU;$?RLFeh#>s`Lo#%vS%-T%})oY(#+1ywL9%bR#!WDdA%Q}>{tN6O zT#0GG1w^!c%fr*v-3{yrh?H5u72M~Fu)~7`kh=d~{BYv?R9#<>gM}4%oK>GwT>L`N z%*+gV5Xr17oRN|7Hz)}pG5FqLqbEG!Wzm7%OzG))fkIlN3@+Kx3ALOo#mK;`zp63# zXVN9d_lWOiKR?;=zK@Q@_C?M0T0ZIeN|P;Np(}6l^68JUCfTgih+Yf?&SLnYz=#e$ zJFE5z4&XWjRw!4XAIHb^|M_#rMX~T}&10jyL(8~Y)2z1ow56>ry_Dl1Hk=D}^|@>X zGYfMzE$whbI7D3Bq^E~RMP)c5;`OIb1E__&Wqrm{aXUTHuzk0a)^FQGC(4Z1jB&Tp zu;=tAtv9rrnZ7BT8&XMLN?2G?YVR(Ed_<_G@MAzyenMSWT)TbjTvL!zbLH0z8ueO_ zFmzObD%?}q$jDYcGQYVQ*3yz25>nOD5>r-|8KqzWg?DuO_2s|3z_33C>8Id%q@|Iy z>;h8~ZL*jxfO-J87YLM{=15&fzbcVA0eA~DGLQ+af}#iq1zGg-=YK(TzjIC!Ofi_f zCZKp*``OVGE+9Zg+bGN)sdhxLw1b>hp z0}wG;G1`&vL>?vIeG8J16ga-V!W=oo-$L`ZFgK5r53Y}xfBl`tfdQg`u=Pc%@z?z? z-uCm8$+Cz3SRo-<$JS$aHWC(@3xv!z@dM0$6%B?$?#H~RSg1ly4yt2%Gp(U# z9OZxtoGxSJBw2#r?XDyE8TE2pEQH~lveWoQxSrz2Zz2&ti}UE{U%g_W%PY<@u(L`{ ziyog{5NK70yv-rQd95`ynGz6qVzctT#TfoP@XZMnZ{c^g`1M=~&1)UvNGcf^2)Bel zmY0|1D+3+$(`n^bSLbO^cYAxjq@+lPgh*{#1%J`f?CbB* zw+enQt|>fB@ed~QZq*C_CziGECkXm*LPkbp9RU&1@k;AGNZ^a{z79w9_4NfMoscjd zoQ($CgYiK@LCyk{Fz16O##hyq7QM8ys@l z+Jq0$mB|bY)(KhEyzY6T)n8;o<3t49yU5u^i0&WjxI{Zt(f#e@grOqB7D|s8!fTVIN$F z5re#HUcN8P+>2LD;Quwti=RozMD-F1dk^7ljwEzGKIm41+1Lf&`+(crrGrqHY)T4> zwOLjgIv4FvkAz_aW#lTybtV~d_n63CuX9XOAm|1T^g23NmzNsJ$yDs@gN}|HpFW*V zeNdkp8e!vjmstCNz_c=*go$nmm9x43)BpGKAUTXUN{R7B07T=p&SGw^R8m??P>^IX zS-!#Lsm5#%Bs5#!4+3n|-yjChIBd87^2kV!w%2v#OuVYDJ6}OCu>EQ6KvhDi^m*-I z(|;CQ5f(o)HGAdwb!=?pXzw_qD;fAiggbjR{)?}6W${`D4H{yHXjT?56#h%_fV{AN zCLrhi1>s@MHm}BS-w;2-5DGZaKlXnbE7xoP3rG@8jg6h<;8Gj_8_}jw3Ix6Hcv@E4 zeMLS~KNL1L-h%8U!1FBvZbD43fZORS?XJGQXN?l*FlWG&1d9jKbYW-QF@UxU@S=YJ zT%da`V6?j#+2g&}&FM_BIwA*XG+p?{RNDNYEalCnzG*$`m4+JW@l&2VTvS$9*YEQ4 zzl>vwT%>Yi!uw*i9>?_F@EskL$jojg_tQdQ!97q|NwMe5^fUo6v6-=PPfyRy?Jc<9 z2T%+qW@cSN-1)-RacV-5U5WM*YlKWZFLMRu1{aL)LoRDL@%V;DrEILZJw3T%;xqB_ zauMO5kdTX+nFk8g!9tOsKREykfX)3ij_~;~vcfXbYv^nZd^C>0@j_mrRL9?mO996J z*8|3g@r?d}iHXSoBud|$*Ixcqb2e+MGh4DWVj;grdzP62iTV85TPQsP7H4c{Bw9jg zXlnj^7RB#JL8$;Ks5FPm&BW0h5wB$;V*fEcpTkhNF>o6|lo;G;CE0;z=Ta3sA1oEO{Wrc1ya@Y7q@M951>NGRo|AW~UbyzxxAyQ(OrazUSR zk>?eGR7s|{F0Z3HJdpW5WnUf`?sJtHt;a1=MLf3urufS)Hq9C)akkY$@b9ykO?|G| zMtIzko}O->Z>sYg!pVtiI}d`ZMMXvRC@9yA3v62mbKr~gDJN^mz5CB+seK$^P_`<;@WDf~#F3ZU4bpB~U5v zK?YW>Oxwc7CgKBj+bRed>(_-s@m^v5t+83G1D2%*+%&Xt2XKuPO9SOQ2V43ZL|jbF zoJUhkObp};UuGej`72&@z2G?g^C#=BxI{GOmJv!%PY>9;|H!*e4h|o`W`Ph20B0gj zA?eoD)^;WZD5nd1FDxz!LLew1Ux_S>>M>ne5*o^1yk}Ef<6~pks*&{ee`HK^NnEY9 z*%7g+dQ3jxx#6*iA1WUw@}yo|cljtX6&3)$W%yTm(jainHbUDHa9d!%8obD&$xLRO zlai8>ZD=UjJ5A*1ci?1uG4)0hoC+vV;eTtwR56c*h1EDeuX}u4^y7!&_wUkGRd)gc z$L#El8XDCVek6sVy;r{z3115eKb#7`kcgCoaS=!ltQ=7)=1Yb(js+;2UpsQVW&S%} zd(`_!3Kjrom`tOGUPK|(j{u5lj#(lnrJbcEQVqg0(4qQku?!B+&CUv29e#Y)fOll+ zW?^K8WA&|mua?igj>_u@*H|u&kN`9=s#{y(7Z(raHE@ERiK(f@7I*HBB(MnfuRhFy z!W0`5!^H)4b#Va=-B-W=zyMKGU5$^6`wR5+Fgh7}QJDQ}RW-F0ke~*$10n+`XB>dB ziC|1ZK@sjjj*pMb0C)vMpl3WfI$FmCMGP=?J4%AUiggW3N=sV;{((InsG~+kn*bj3 z50wK5R@h>PN3NXjNoo?R=+l4-U z%?-IX=DT%mUvu=lv1q!umdmsz|I+!_l;7ZNC5(eNQ&_HRt7m_o@=czc)8zDF$jn9s z4C3DjP(X+Ly%$e$DD14|)r7NJeGq02>Kjv7L$h&J)s*6Ta7Y!$xLwc^easH&>9 zYjEb@Y zh%f{<(E3bHPF8RuKoEApce;+MeGSsE9}8jj)q(3MJkL>XK!zPLr;84REx*2!y&MIgcp$~a{(50yVIaS#PyHLNrd?t7L{f`jGoaw6 zPpCj^a<`^pA*ozQo%PytlLavwXSy9O1fxw$n*)*pXe-c|5OiZw9YFdQepPiqtH*dbjFvSK5e1Jn?XJ_wl{1ARBjvPb>48Xa$ zxW|HSuPZ>NhD-oG95KnlZT&}7RB$L7g-rgGA|28Vp#VBo*A}2sE1{q&iix1s7 zTsqIg%}q>PJU%gTwR_I~*))n0>HrTnx2RP@2Lr-X7EsLo0wNUmowzghWp6!U!X>Y; z&s=5C-jAN|hR^Oqhm=PpEo}2L=$H@QZ~C~kZ_H&DpKJYEw{FPfj%p2sSQv-j<+qR`Jv2 zZJ&?z_eiP14)Z??FvSUAk5zp5PWa>$K0C1Lnionf)n8b} z_DIX4FH`2m`i=Z#TX%n;;fsQh#=(Uo`_sEXUtcAM)1#(qJ=^7)os~0_?pS@PI?g#Y z8k(wwsR;;BxP5@0lT@h3c}XgB67(k;y|EsI1dB>Hxrd({+;A>gz7mh!%AkGrG$vWQj>;AW9 zzUWxZm=R9*nwhbyfguY-C<^!WqLvBbFo<2~Oc_36nZ2MyA*@5jzFh3` zV!Q~(xFMyhoAGp&t`|t~E}4V1evA9~vY*EqS+omGuWSArhK=d)7kAHA>05u&v(fd! zXVuiCrIo&XQB|BEQSfGBVx~Pu`>BI(WPE@lf*Y^d?+<+iM@)mHGR7%k&{1+UZOUXd zC^BLb{J;t{Y%;Gr8YH~pD6R7*&5;fs)5Q|0WZBs2hkv^FV;f?Noa1|(c!=iobmE*8 zJ%TFYE7DFN5U&SBz=`*C8P@UgT-$dUp9p*n?3N0b0e1{@@z$|xk;9}+OSNr3`EGki zfGNx`IM{(V<2_WkisyOy=n+RO@r(!`Ui!@Nk*=(G+8BGk)^n72J5_ac;&^2UOGBA` z`88Ya^~LgLSudhMKT`!*3xUD`W()wN z@dWJttCLlbgJ1m@#0RP$J^_IkV(}CmXAWqm?6W}m^#MAd2vB%3ggif!uvwMPC4b)h z5d5j~zKsvvALEHZAbJX~^Txoru82YKa(P5)7RcUYQD z<#lC!Vd6jyypbIgWSLFW2GCPvA*k0BK?Zd`f)(Xf1+5(+p-}(Vdv4O|8SEEy7tcB7 z$!vDTbEft}AnpuM$2Z@#%z(^f;_hxN7LXhh(@;?n9uNRZU3{U1vzVB-&5?1@d5lL1 zeTURdrsd=9ZYURC%oe9Std&6eMWB*g*4DZ$oRfmSm61R_jp4{Xx2CTUFJ)wqkhfUv zm(3R&9P8U1kKlSe$6paI>1#SRn`z~bB-k!86-4F1=)Z!~!S4{(av}bEPpI?hFqH2S zx>dMb(0J@SWN~6u)?DAvT0gOQl(@9w1$R2vJb62ISLV^@yEkb*n9Vat`(f3iMH~0R z7uS~~Hq?p|pI#6PlS*sL2(c*7DSFk<8{WoG!@K(2>{d<18O>y8abaQq_ixb2JsDH? zU*3y+U99}?Y=3riUeDQ5wo?wPt*Y1V*i%KCs)2D&lu?|^6M)MT8mUf^|r zpC=tjI)U+My>G54=jbtVaOg&I`+YM@;qA5ef7g_A$@Z2uVKYy{>i1V~&;2)#KIF>5 zIQhSVZ*we^d+B+y>nwu0O6OY$q-SbsYH$#vq&Yl%ctq>Lccr0OdD-b(PlNETgh+V- z&Q)P^Gr6kGb|(Z5FDNfBFDxvq_^%^TdPYs1xYbi#S3*KTc?!NF%*Nm7v@?`4W+WFL zb4f&md@w5{tR9``WrY?vDU}tc%-BdEjd7fe`SbGw69F|wFaB3WBvk=L#z?*|pQ?LP z=}cbE60z}ir;bz)*{BQor;O~Im8ywYwHwrK4vkgw<_C}58dPiDrycE^W$~AI^f}CM zkGfUsG!28p8f}o4c#u|okQQfpt)PeI=Ei3;3|FE9qob`96DuwYx2e|pgw#m+@$u1X>A>@l z-fw?v>T6XK5+wtI$qx>4YDH^irYVjNVfgs#3f1S%c)}PYp8#qFr2WC=HfQu_(@oju z$KnMfG}^_Wo)#Bd1+n)qt2mqePZ@_Fjs?X8kY;o5tHLPimHPXnN))4n4}5MYbExZ5 zDYZAeonuE7%2TP+@c+U=qx@HDw}V-owaf+B;*BKgtGc$xg^jT?_*uaxDxZc}roRxn z6c3Q~4Msb=yTG*of@BlsJQc?vG6MQ05Nk^;*S&{+FrbNB`Di62(AfCZ)pay$d=?#r zGN`;EL4Hh1;j4o^LQ)(qky=dRyXmPMcTZwMf)36#SL)r2zM54|%?`tvFIr!HPCI{3 z=7DjKwxbs4SZ_Zw#uG>*2EsxG8H||Xq%i2VXjJbu^r5};lgRxtVo9TX##Oxi!6ODG zJ2$^a?HgXKtXUu5Bn$m1{#HosMC*r?)Zpfl zQ5^2$8y^(zJUmm$$eo-VE8+{N$|wyBiVt=-m6f#Le@KDlt9eM?rmoKkhY~UV_K@v( z0&2%=G)^U^&Wzf&(IlIfmyG(qFrr?wyvOz}yE~$a@Rn(fV!!4)`?m6Jj`eS_>l(e5 zQdC)2Irg$!@7=DB2Z>Nu`N@HAz22XLTKV9$OQrJuuXZm9{h>KLj7b#ZH8(ed_RZ4DYG`2K^!z-t+2NiA_28Cr6N99&n^9?SxU#7! zfBVUE-(Ti#=cL7o z{B`5@pT$O~MHFy-Mn6A1e5g>o5yw9*&R1rfcXOFnN^>DUwO?XiN)-sXezM@i^TuvH zn5}(|emOI3e}Aw|bBTqCM)%I4nOK=XivTl*d46GGVSN0vs%m^@X4ISW*4Eb2($daZ za9HXQBxonuLWgOR)t5ubDXDy2(U(P86np;(u&MY~!N0y}_DM*-<0i>vO0KGEE)xFH zbZ)S9lv}rf?n=(h`0Q^l*!uUc1j-M9ip!7nD(KuSId(yhIl2AWcF2jMoPsnNT%hU{#lndQ)s5Fl(g*xPfy zdGl!=HfOtZ6Q;1{ja7bYxs%vD(`tb0D*utAyaIO^@^V2SAj~Y{jsQhZz`v7`i5cVK za%N@{4<9!=F=2LQ+`gh#PF~j9%4XXY&0P~;AL3PHbAJ_~UgEN04MmCdcDqdHO=CEq zrZqvAkH5Y79y{-5Y(Yka_`)wSBv$u*q*o=MCGCP38wT2^S)$7d@lDUZ4Yp$%F8Xyg zLJ?njpVIeBA2nsO;uING38dLZqet9WZE+%FL~*zrF-T!)TxG^x5Wq)7=7fXJMX}_G zNvzYJwu4ZpK}<#g7|emF_^nh?NePdD;9(NthDhWf1U|Qk>1XfE#9$$~ZcD#E*lDGt zzIcK0;O$kTqDuFtR`jB_2CzlWoE-bK1p}sgEXjUXl%Q7^V5j;)`nv~$c%NZdroWfK z`wJ(elEfWC{9s%4NUB>>|AhJ)TJ3*&=Tc4Gu5#;IJ)#iS5@lQczCqH6+WwWb@7!r~ zRPA;50iCX-$RvM-4E0b`z#R2H6~gnLZ>O9dkC|3~CXgqr^wl3aUhA>=`nUGopQHP@ zxU~)+43OoE;-gi`Kck1I!TJB`LyBp((ClM*D=QY}0Kbxdjk3P}f2padfq{WQ7X+xW ze=qb5sr%xoc+K#c7MkJD9UWiP^{$kXm%hP)r~Yba02=&QDmX}a-t(yFsV8QnNarYx zU}7nDetqrjP0djrmkaTb!*U@&+o};IyXGFX2$xk_jL6reE{_y?~>@~O9 zh;w6>QERyyB$~H;stKFyop=;S81{9L(>LVTx~A`~_xP~n$H#4xZxl6d*7^C~?QDj{ z5Udk3cP~fH@O_xw6td_ft$GpFdt~-ULO4$pW-bcu87TQD(_x>80#ne)Vj-b<`TP6( zA$j>B2)98*zxlR;_$%_`e~8b%jk!^10AUDB zgnGqZVc+GinS78{(9ytmjoKQNzRG$3hdj@Yci}M{`$5*}sXNvy3YS2)-RPoQi(bTV z`BeDsjn8GiJ4aNLZWQGyJ_fT)%3c*T_opHRdY0V>foA7Y0ga}#l#PR9Y-9wmh%j() z%v@vuY+R?^=W+Np3{C9q+~St~@`b#e)}H2LPGoRdA-}9`#~Gp~vVL_?-6&y>FzPbU zU`q-@qC)@JyUxx`!U%}e;$o}oDxP0lkHH5GdEUlMc|hdxBc((C>*u=Y=(y_+4puKo z%9cD<%n(bCUFNH!;h;lI7|_uOMy8@2wxqaac0e;l#ZD`TU;i*LJ4p4nc4J*p)y-dd z;*ZnpdSt)EgCpCGi{PoSv&e<$)n+<`9CR^ z&gm0q1A6>rYKWd=*wZmLfTbQB8d?RQ&<62wyc6EC#lU$BzZg%8ZTAa_4r@?Q?hmWL zSe;wI_=a_OrB6am@ny75kGD~pag3e^<<#Uu6PTGzoUSQ z_y6w$eksd9Zv!_ImJYNGk~IZ>iaN5BQU3Nd5*0P|XyS;sv@~Y~su2g9jj8paLXL^4 zF(*m5!6Xw;78#m=ahSHN#L^_wwaKzDmBiK6HQo4m8#ZndQ+S#rmj2l)V+4%o4Y{vv z2n0$s(_;&*p?K~n16(*jw*hc5TIBMqEE+mG7hnzmFRVPcL_8U})SKy8HrV=Swy|BB zd-T(*&eGx1^B_!?U_-MB?w8)~E8~X$c;-d5cCu+?U|sTxjg6hv`oEk_aK(m8tgfoj z$ob+Tf^-hUA7v!G`0?Q*6##YPtbs51CzuIx^ zLP+kZxBKpK_8gM(jHV8Px143_mKr8BRKx_jcb$*aaupL)=@e1tR#tT5lJ(Tp(dy_q zI5@z?*35EcWhD^5!DEcgfQ786T3otPtNjj_dpW`VEjfMS%(RHT)M(?s#`awwkXnBI zHa9~mvMZJ;;7vnv z7zw|7Qk+ypO5>%Glw1y|{5nFjY@t+yIhYB+r_Xr`E(O8eE@NMORMg4FHVcETOk2Kq z>H#M@$!(B0US^<~w1=s;_v0s5DMt)pl(oLx02J1HadmYdPs>XWF7@jnSY(DL(&iDx2=8IvS$M*L+SG5ro9SRG&6abAI!QQVlpP z!^+!Nva>$+HH_lGwlM9Fj*lg5;oy4*kb}776YvMn&rMCiUZZDc2awfEqn&RE)ULl>~1MkB0G0+r~A=9J&p5?VS1UqXvjeC3NX~Zytn{Y-r$C)qc5}B?N5N;@V&>kv8GYgB$V(a z+ZZ>E)fh4oZ7r=Kea_EtHRB0l@M*|iI)6VJVaeUA?b9jqJ1fR|^jp=SYd_I+kIr zkKMXlmrKXJ%Aqq$6|>8TTK@aT6^(S@r9^PQ4kkbp6qK?bL&tBiQp-6*^Ca)$f)i)! zYij-fq*G^=;uk!vw5Ajl^)&E(RaY01lY=TMs-mI_S{;l>M|VAVNhsspZDC;nHV1lrNePxGoRyGqRQd z05F)!OG~ln-*NWRP{NEbgab?;CLBM1@T+ZXgq4&eMMq~O5XrvSFf?Qy9NM*N7(m&J z;e0B_!7dYVG^VG3DgPXSBQOv%Ilv-VTel^>~vFt$vC!FpHKSd+U{;Ug!VE^ zbm5`6tvX=gZ?ZzNiKc}PKGAdkGp3!@AIfRpuV+?f)uJ=0$tbLiEDBi}7}(g$@=K4b z?Wie9H8eFBrj~{DR`0iF9NOn<*=^%jpU}4!!xO(CJ}YR*a(q^NID8N!-3UXEeg98+ z(zK2>0y~MhBv|^y*vu?Ppp1tMBxK>>85@wiS5s5_{25pOtSfJ*eVJgsRR0<#ccty_ z*z&8{CBg_!m4i#Fycm57-kTk~ySqzBAnKxlHwA4y@a`+)Csi(_q=3GRT0z0c+Z!Y& zNzu_kl%FgeqNt3@-h+k@!jIbhUxvJb zoOHH_2ie{}vy{vb6iNzgX>OkRdAU5(a~!881p>i$D$=cZT48O0Bc404G`z7AYs?eWn2paB}x4IIQaRyOHq)XQ!TrKc;+O-;Y|_FVcS|3r#9X1twbo1YOh-q-p! zEco+LYkx1LsiL#81SMA*^c0=r0RaJkt(`R~2nf~SoelLeZ6LR|^@feQoO9{eyeNgg z%K9%i`$ugD`QoOGt*=pae%a_KY-1y!fe(k6SO*IWtO%jFxTn>@>G*ip_wS>Tk*N2q zdw&P)4c;w-R%B&v_==*swt87$`DE&YPucmeG~2~OV#nWf|GudR7O;JEHhT|WI!G2o zvQ((~`22ex(Vn>K4?--EOUdl}6Jrp$eZkTA4@0_@?2EG;=O#Inm}uZwC^vQ z^72kQ9h;++=iJ8p*L_h;JK}eCS~*(91R!PwF}85`&kkr7NcE7)I>_K~6aa<-YZg}- z7?FWz6PPty-TrLQh*vk_UA=M8+L>X+(}3xzeS34>&pO|PaD=_2Q)yCgcJ|5tyEQ9o zN92IC)bE4eZjH?h4D^#$mUqOuBd1rLRGnVrpyXG2aozR9%h~SJ`r4E3P^QIrN9%w& ztB`a1HYJH0M+~+_F`o37NDUds$s~?5IWbXEWfd{0roLX1lZk*wPDpzPfv1hLq054o z5X0V_gS!5jmNvu~0}LqOMxy3!+SdJ`@i=)&!m5Q#7pAE4>?hia8<5kr;A&@o=)A}{ zNt9r)0xGI!0s>VV8$TQ!XLoiw|2LtOmcGa}<-_zp&hkt#5tcD$5DvwYGG`K-ss=&n zKB6GSB4W}x0XCb$U9PG~ma!iYcw-j2p|%D-{kP_hzWkp*B_t(ZCy5#STOqKI!Ap+y zz&HXHsZmBf@=w@FOr_flU(i>#e|v^LY}&3?s0WL+#mmd!r;@fdrrnPYDYt6l)>#Vz zuOxF)uCHchI&yMcA3xIZ^OMWSBw=hBo7MBY8;i$rmCYAw5lnjOWUW~0&%fSI1aFOm z*Tsm330xgn;o#^c*}@Y>I~Sidf|7M%E?W=L^e!qbjc~av1co2bjWEYs+S%DzTLV@T zGYw7W*cf=v)ltBwN4)F|*q@iW-T94mfZwa)0$$q?&xaU~7PP^%w*%k=^s9US!E;21?!Q%g(X zoQ^FGYLnR_c@<>d?(POOG@!dL!oV;@LejLf1V;lfnt%j~(DL%q@tRo54UG&KPM7W4 zW3b^G<~j#+_51tm!)ASt{;#?C*}!jD6FS8P4e0o9CO>4@i#EAy7O2i9nDje9fk%jC z$eVWMHytUJnlsTI$-IqA?cXIJAe@_-apF!8N5aA;Obfj`T$PO&&tZc*75S?%SQL}9RNRO z*KYDUTPS^6R&~=G+NC=MsD2{(@Q@877}G(3lO*EeqD4q379ZcLsMx()f;wLmZ^Q!}@tWW@($*FMUORe9 zSoQc_BHO{*u3iOV**VnO7S3g)9Xk8~z*9AGTpf@k5+r)?Dk=i!E$9fezd<_&Ui%m~x5X ztf>0xh>e32jO}CyDXK!Yi@lH-ja!E8c6Vz6;5I2K zZVcZY-UmY-1f5-7yPiIUh7Y&aS~bdRyvSnPY7)E-mUvOr)UX`3H|@q%cA7aeJ3Bi) zEe4UzzDexp_yC~5;2sQ#zEYf?L+4IQ(qYFwX_Tz7tJY_%L^4EsWDNPD0xTh2#bBq0FF zZ;i2qt^fm?6SU;q)Np^;a#KSnR(}VeI)Nh?4-b!)mKF&Ksima_m=M`O+4u$CxEmT8 z0+aPB$h-ivI5E36pYQE{a`BvRD|`4EgI*0h9X(eJp>dh<&WLhBk&@WK+sR50?Y9;% znCnBvIXF*~NJ5JaaY>*=?~_nnT%8Gs2*d&c`4<+7Kr4aq+`OQGZ+=lSD43X#s7t;3 z2s<*J2C9-xYjrg2*u8&TP{`J^5wUztco+tLGz%gF^S+YJA9(bDn2SgcN=tT5&P&;a zU%&jk0Z1qw21D9(1U}mH?;F%Z8b{8fn{pZ2pKMkoN{P{ju%zHj>`bT!$aYvOYe^2J zxy$ZKh#E60s#!pL{pr&W3=Dyt!vz{6;U2N4R}rO{i^B+$H2k6w z&N65JNx{;LQr^pcPX;3<2Pl2OWG*d@BtuO>A)7O?w6t^xUQm~l0|lvbWe5(Z$=KT5 zZ1%c61ImhgX1jfW!i6V#E7)f%oT2mjvCRvWTNd<)AEFTQp1%^A%%Yri+){t{d6KK5 zqB8KWn1lq<0n#6&>#i0Q5;qtLRP$o~rPqR7M1%$@DPmh&!q1<8r3Y5+ncMtzPeVh| z;2ut4@F7g+AVF3io%t*AIXTCDFp z)oXIzJ9BA!;q}09)7wU@^r$p>YNT3Ue<50zMAv2>+dSKbJ76E^`G~Ug+@@9(uD01Q zCu_LMg(3Lgo`+QT-zHdHLv~75D3kW`*cezgR>0Bx`D%WC{zJ>SYRwv2m(&^}-fMSh z&c34JqK;E!QL_EtQa?flpiqsZl)EYOEzzgv)LFqo1Efm2Y^hrY49yu3`~gtC4BM|a+T z|GlE8?*qjbgZSLc>`!_+MIj+97Z(#c2yp+D4v+XFA)ProYHDc)?IO|G3->@aG|9C{dH4Q$ zT7Ww8jC?O)>un5#pW!XNZ1x%k5Js1x_5#{3I0uus)Bk&FZ z{V9Y2jNm*xJaKVx#>U3LLW95r`c6;aFaQqBPRzB9nS)*1rS~o}zYF{huW89vp})OL zcW&|$cG+#n%9-jY?2F4B#jYbngl$_6)DxRwOx5(HZAk3|9<9f>06Ogk)W^T5!$G zB$v03x&%QqSHn15%`L$z2qSE@Usu&@ICfk)OvS7@;XAix{$OUD{`KpN3O9uoapxdsJC&KguuP_W_-T-6E)2>8W;i4_nv z_5Xq|2i)Qs;JteF%KrY!2E;zVvwvsHEiPJ80eUJM`;?tlUjzTsAtu0VphxD$!S4KR(9BBqbmq09^z%wL$|e7->3wApfBa5AfttER64w z(k1eGIf5x6ucj8=-VW4)?C|i0=;+9wKbxzafOpPaV?MkL!b=auD_#MmhArH-^7Ci% z{l5F>FTLrO$N5-e(GvW4^z-^aV?dV8eq(3v2{J|E$ppN-yr8=RhsBQ{Kg2YJPqz;5 z84^D)syEE}%9h^5r@UTI-Fg|HloW^g>}AtPP>emfLATW8#c=dm+w%0Uq~wg_-AyB) zpK>svPhoQ#8Htdwu?VG4{CX~E-yC&l_*UrmJjQx7%2cdwLj7E@v(%`Y=i(;PL%pLX zYqZBJ7-YAiF9160Lf^@V7gHACR0Y0j1_rc4?@ynQP9aHNAm70pkD<0YG4T?!y|I!1 zmyfr%H|RG1lLl!i1eqi>1XA#l4u~lYuEBaywbm3sC8;2{e@&4M4oWeM3v&~b?X|VF z{e6sz{euH?P@qXkLrLC%LmH&p!Qlu7!MZx4reIplmlAkEX*~n{G&d=~-@XN)6BPvo zd~a`32L~`3m+Y%pQVa{fr)wCdb4rSt>nfRdvvta+z*(e z(o(~`|Bt4x46CZ^x~99k^UxvPC4ETg6zMJr>FyK`As`_olF}s&64KqB(jlON@-6S@ z{r+)r@q@khS~JERV@$Pc2sD|~c<%Ml3p>Z-#*p2TRwjK8ej2qR1PS@Wo^|;If6Z9h+#O^P8hTw5vC!c**_J!}iGAzNo$gi!6nQSM`ylwC@**r2uImd6D4$U7U>aqJ9q@hl z66nm*pwt6t1-Q2Wa13M?z-R?m6j&;Q_6$&4fe4~!2IzBpdf;pk{ad|W_hK#~gdliD zuA8^R#Vjo^gApeDyFRLL#bRU>6ojfiCOA`kx$r-LQK@K@3L<$V2TYI`HlC8!9H@cT z)pcE8pLJv}%)$cD2+*|wDib9~WlB%a+Ntx;%rf2}*Rd)6MiLw~M~ldJuR6sTp|IDl z#|r6f{X`Ga=BCX3eR)<_3x&rHY!7-P8glA~(;(>8ClR$a{w#^C)khwa(#}WFLnB> ze_<^1jcLP$$Z=>kfDWFn`}O3n0=o(7&mvC7pK#5#xY*dx+fhmI+A)OC+Q;KlL7p)s z3OuOh;xgw9fBa){P{_)~#lhxKQl@l!-M4^KUb*YP^avtvW8iC7zv*A*O<#4-|9v3q zP!7#Fkn5|bm7T2f7oZdsZDn;Lv)twt%3814B=yBL?n9#a=w@tMGBvY$kOKK zCLlIwl~z`Agq9W;7e_=OZk1AVo0wiAH1`z1QCkQvEdv1`4iW=0H#R0_0=Vi435cV4 zAVmT7L_q;A7Wy4%Mkt~h!qJs#l#d`D3PrK{*2Ae>c0mzqe>T z4b3Q7M5F{W`UXDUUWv(xoE%(d-@j*PR@V9}Pk9nrPgvOZeyVwRy!oyPgD3+tgf<&Y z&s)cXhYS9sq-@q?1AUTJ+0!-)#Jk1MU6#Q=@Nf3tW#{IFCD8^CKVGXIoUf0yY}$7H zVqP%wg;(t`{gU}>!DMXF%&MN-0)Zv)kovSe)<6_WzzrYrXOHlpWz&*!`*MHuNt~+n z_q;(}$v4th)?HU93w3X$Z6`ukn=ihSn#|9W&Kc8jpQZ6-J>@ji3MrOxLB$+<0S1x38a)uh8IOhNhw=bsJ1lSuU=u6JmN1{K(XQ;VC@ zpf^awue0Vgl-V+4;HUM?;vzwj&!KTe->Pb_VCR+E=%3d!P*nx2dOfp^iEr$jkdoRD zABsDJGAqB}TP8%&>;DaAmtySt0YcYUtq4g{l8!Zofc;r?tW*dnl(9tc)&PqF`!Chl;EZ^M zr{$vMx2uMU<1&A5HSRb^=Qi$pYM9%foE*>S=mZ4?jwL2q0}1ihfyCP%pYrno`1vCz zhwZSZHrrDkW-kUN59Z-nloTs<-Y1DwPhrY4leoKLjr&{A$LP`d1OoB4evk2wF)C$= ziK%I7+S;?u4#hZF_B{gOb}jZBkwCH=NA2<@oUSf4JG+k&hzlCIJ?=y}!ublF0;|7}g&vf*;#N7g}P`Y+vBb4o3U^mGE3|M~y*jVrRE3iz?99U-EwW zGF$pd;*55B3&!NK_}NEplO*xL=}cc=9WG5?(4h92yOq*lDG-cus)Csm7;R@EF4S*> z9}f-=z=R6K#)^uH&~#Q7T0Q?=0O2!L`$ESrHC-Ou=3aJb`}|tryT&|e#WlV0X(N_- z=&8nYIyZcn>OAs{nF~s0d&t!e;l*vYCUjq?Hyn(!J=$@(*xLI39X%718=E9)xPp#= zaocQqVIdl$gC23}XRpFxyXF<8$f$u@D9@3RrM%gg5vf1*OWwM-~4?>&P<)XyYF?YsZv)wt+U zGaKy=nlkB8$U?^yRywQN>E6z@@nNf!n34gUT)clYZLDTtC`6$)1VV zBOkqM2NYpHv&eA4h{wg{H9b9(k&%<1&?ja~}j*lc* z3~37#y;yT+-&)}_R`ZgEBpQEWA{@)ZNH|p#UVLa4_;&8 zcK5*O-J7al+*~8s@a4IIsw(T?-`_*tp-@F->!@J?q>NXYpdJ39zuX+tw6P2|p^b~n zhDLla@Xp9NMlL3+yxdUV&Vba@9q*&VShLV6WcCj}Hka9&|FN!}*0k4K#*KMdI_N=h zwC$KR29a4Z!%F~0OsA>n3Q@^$gMR4)HxJL$(9qEM_;_z`FE}e>V^}HLN=izgi(uL- z)(r#@ENCF`O^Yg47%JKy+x1(1RQ))rrtqcwxeVB0O+uF>`y?Ub_2AAd(jxu8F%LNDNzFLo$U?Rbl+@ID2;rsH2}7pEvH zPe<;d!O)G}n`Xa)Egy5uvMrR9LZHwY{uLQ5k{r(Pm3dzkxJ#xEZ`r9yonztRNH16NXT}l>xE$^3;JG=UJVJRzXTv4HM zcLz{ACItl^R+bR<1sn7!PLi*Sz2*_Txh$-{1%gvvD^Na3>KnYje}r{()3dYVT;Q1> zuar2?y-xIP10|YEKfpblNI&sJMP@NEk+nrdns1jUknjSxh8e`mv{VbTt}!pqfSw`^ zjj^Byp3CZisB$DrV~1 zE?oq`<(1%HrPPN0y`Ne_miXI~S39$?lYR}us+~oCYeK$77YiocTwnKSLs(dbKqI24 zh=Pjx00^dje)J`cUb9 zqv&whO+l}YDg!cq$!Cy{4-ybqo#JleLL{;=Wzu)88IPMGKwP5Z<^V29H2;h+A4lK z!-JqtVSX?)jxVH6)HWAlV{^`fE^R;k#r2d$5Z>tc)q{v`ZSR_VhK@aqEk;5fN4K)7suH1`1M>{hpG52&q_J!ORS>EMA$cj;>3x(Zhdy#;=uS2hbySVZE0oR`0gNNA*mEQD1$y z7sO4%v3sNc>76z<2jcq4&aBus#q46nvDCfhnS!*vGBw!buRq__-cr+0$2pGgit#2d z>bKHPew>_)3OvjNXq{UQ8FQS9L@Wz>C9aR0eu;e8XHdV(M3{#3yYj z=_tu9LZ#7%ks|M~a76|i7)E9#3)joYKnG(`#Hq#k48Y(@Nv#v$y}G|09~-k45K1P$ z#Xv-iPI?!k6zt0FFVbh+M67593Y7kbi%I1+>Tt;F3QvnQ;NSKV&~4Vd#KqHqqTdM1 zPI`M~F))x($ztM$yvR6Q;>8jvF0NY?ptxE0;U(R_-;|~s`QlR-FRsG~wLe*~xD5z? zqE>y|;ZM9f_;6nrIpVYN>n-GB-KwHOs`Xo9Q~sivQ%mDsZ`0O4@uC2)RVKDlcA7mo zdpL_$Y0HlvUJP8Icw-O{AbP<=M+axK3n+yF{{9F|?G0$bQAvO>z^X-?@dk`h!us)L z`p~xnz0?U2Cv^+DX7n_wN^LhE3p;zQdIwzx*NGU*n4VC0wIv_>ipQ zCz%ET%U|!rBXRt|(IA$SU%7Y90}rjQEAG95zLtCh^!CF-5ettW(|b_;CrJHoByEOg zOj#lF8FV~7Sf=h@ocHj;6(NzybRxNzGJOvCGDY6)%p)l!^sioo7i#Oizyny5yK|YoBvMCAd<+;!GE@MM^Q7ct6iy@& ztbXD?h7anxb6f!EzYEL5m+aei2kQw=K@Tp%l}p7PW@)rc2J_8<>ifJG1~>Km>0dsw zq;htd&&Xg>kZ+uyU(e0`#|PnH-=ZwYti)p@rt70ee}furq-{3%Ie4dGObZE0p^s9# zv-1UCxQ}dk>M*}BAmn3t@!8}<-lMqsY8CrZ89h^n=ujuOSwmvGv>8F^^Lp4G!Ae{N zdi_LG-FzpO2=v0e8&PtlnfY2Hf8Vh?jg${NksUFRLYpzivoELNrK1({P1P%OsUfbc zM;7^PwTwL=PS0(j0CdAWXx}BFnF?Nfkiz)@j_KKMWtzfztN-gdc*y%JW^x!&rMZNF zprDPNEz0lT#-P9&9acuiMCA5wZ)YC(+B@gIwIL}W8;*c50Mx4&YrG4K&$!uTH*m9_ zEzHjY8DrV+-_6ypq#Gbm-L>TwTn;zH2q6y4$_5P5rusfI5BC%Y^H4pgjf;$bPr#&9 zLLs9u1W|84tN_l}ee1oOR4WF<$cpiSU4eQ$8xC|basZN2AJN~0uAAG*zm@lp%Ec3<*Rsd@Hl~C%LNXy{CePszdQfA&B`7G=cXU;% zt)7&)S5mSEfdGRWMEl0EL@Y&}+4S8rQEC`#DMH&CAZN>iJ?2JtSZy=4{C<$xM}r zd`hTFF+VtJrMb%fOuTb4TYHHb@t$`|Yvv_tpojn;)wkgdYVUVRkayF;U$SiGq<)&$ z)f7&rHuusD%9IgG{%#E|Jnz-`g)e~Zlwu~_He`fJwU%nQ zOUik7BrU63%Zz5XGYtMfc+-5U)pjbqygwVYxE zON*-N*SLe5fg27T6oVwx|6@>XfUg%MHnnc)@aIEn3%ft1QWhiIS?OZKxJIc{WxDSM zB~6F4n6)wSm?Et9DIiri>e}VzizA%U=mUk@zVYRL7Ojf*PcaRFhK89;8b6MqoeQk9 zFMbQ9lDD;*TAI{b90tlBQ4d!?zCC7>DVH!m12MQ;B+NxEUtW1cCBvSer#yhua$V~3 z1(IaF)bG43dMJlRybe$fm;Pr1Ie~-2LyrfjL^`^SLc%emXpCA70{k58oX+<4q0108 zIl`o*fP#?VOl5+VFvP4UJfxDAV991WYgNoU4m{=_`kuWag{6stpy(#O$}HNDv9Uqf zx3$W;jEq&y6_H4ag#{tWdcDsNB?2;yUMBegLSdc1XlOFrr=}*dkYp(AqHA<)xMu?0 z^!G-D`czFBGvsPw}-M(L0ur25hPMC~JdiL8-1oHuMdIMS- z!Avz6rWv?V1h>=qtfI8jj4GdH;6`!`pX>XWWQDttM;ic?{6lv);9dcgoT%#q@AxQ( zAQQ5QxOLx*YZYMp7A@dlb$86j7UK0AX~1%`AO-i*Mn0dhr+B-X>_yM+--{loaV|AI z-Rs_d^%Xf8>}epU-sd^0d>{jjZj#>X!)GlktriE$rcU%L;daXTqOhtC$y4?f1f}ok zzxuMpFYaqFMaGF?Kp~c!Ln!E396M(X=sXBBS&xH`KaI*^gq*Y-2-M4KZ>^;MFsX}_ zLoo({#fYcIn>YW^{ivu7L=*W`i-vm7-#U3M|wmHPtZEF+lt# z;aQeRvw3;krIQ`YfrqYEFUcwok2wko-}qS8Y70MDqfJhkz)K%$X+oJfYp4}gXY&HD z8HQ?HXLawIh-FdmP32>!N&&sY zO&+jrasNWPHiu*zHJ^_6E3Ha-Bs2SM{dt`3BMJ#V<+PN$iFv6~A0iYE5^uAQ6;JOn z6;6N^0SK6YE9N6C+!+J3LPkbW+fFfF60T=>LW$PZS162tF)q}=dS6v=eJVExoTeqF zL?y%sHelEtb}tEW+>q|l**r)lTB4YZjtr;t`Umu}zpvu`xUg0xXtJ{p2_y)*mycgr zR_mk`wbk&d@;AMo>U1f5*D4E}`a!U6Ye$#*4}0L2R}W2h)*Xp^mrLRK6u17iv*f z{1uImIzhh-srmGKuh=Cx>k4k`eaYb{kTz@d=0s~Vnl#pTD{G0=!51_rpWM2W&Tu+t*vIQ%|S8NqmXGeF7oHNy}-z1F!;i-uovWNG2sF~IkYjq_byxZ6>@P!tbazzdk=xuTuC#f z*qtqA<;)troL`-r+g$det%Tx0eo@>jUl-un)N62YUge;!8Oln_QFrFd?@$>cu!U6-y767~_jsN3(DU1sn z!9?mQ@FUd2db%jNKSo?fhjdu_VHmBU&9jiGBB+Y_0NQL%${>pHnogz7PIdT@$izb` z)vgyK#aUAnsw$6xuBaU3!qxmH_T9VG{QSgL5%6QSQEhjKtA>V?>gaHBl%IKSPb<<& zBcJdy4lWx^m+DWS&{UFF4*a5hls(waT67$Yr*6VhAFjW;%D|qh8=fOx=3B<0I_a=p z;eBiITez4UNrB|vz10;|3}NXq$xv4>d2S=aY6c;hcyv6naZTNWmo5zWWG2SO9S4DG zzg7D7ouFltwY-h1_6jJ1tSt8svw+(Y55MZ+AuY4YqFR7SJ<<(%5xYngb`9Fmb}CQ=HfwIM7=h6kk`h|w4r8Gu@M4phGKpz@sqU5+Ja(VFC@ zFnWIG1OCMUGqW}W!<&VLOBTA3>LC+B4&CiD>eUIsW=;iNkAO!!3+sJOBsxg22P>-`v)q!xWGld zAp)@ng5gqKUOw`Sy*Bp876TNN5{;!w&gfFzjE^g%ynP{MC%$uEag(q%X{nowl@Ra{ zxXniRYHoB91ztp5bzEq14yu@R?A|7s*Bhm!`4=c8 zOsjL7SN=!)rBWI<0SV;hv{!+=&14*$4Qy2Z&3=WX{g7B*^g_YS`dA|FQ&&Y#GCGUKAv5kwB^m)^Fk|+d9p1@mt{#u zk1sC|s7eX$d7&Or7ql}`8t&|L+84he#oyqHhG0dwRP0VI7o)ud9(m{ay{ka$fsQRoXOeYJ7B1YTDcA$2?0+OvZYf40?(0_r?&+ zJUnQQJ~ATI)?2?wF7s#ZPd1Fl_g<`lr^(H`6d%NWteFz z(0viHX-ib+pP$#y%>0|WDk)K8V^4qaS?19;H6-I9;4;bwoB)C16R3ujNqyw6G5&_k zR+0DEm93sfconS;cvz2r3a7)_`Lyh`{W*WC7q|W~gN|ERIK}81C@Cp?cXINS`jY+* zc769_Y;sa_azq!caSH=l^W)*f^kbNF9fv@+r9nAE8K};{84oO=>tx;{5_g>rUL|Fr z*ETORI=3<-wlpOFV7L>^HhXjyOG(s`GqDXp&d{E8TK+gd&k-2%)GeTXt6c%rtJQCt zOeL}r87YJyM_Kypexv*I4Q@Md^%ZO{Lq(z_!BpfkAoKSDw?**HcNDO3Z~%WBu$Tio zIudVh!Dz}F!laIkExEonU_^p|Pe|bL+KZEm3xswV2zmAA%`g79RCFW1Rt*i{>9Hls zujFSK*~}KUdSrsl^2<|!XKwsjTQMEu?%6i}yIl9xS1}wA(Wbkn(>zpPo&EbEwo1;_ zLI%Epu=rTSHEH=ZjBhmIMd2U#{&If^m;t720cDKq0T-S3YVGft9ag*!LtM>T2fdbN z`qWn+f+l8JpckEOZM%{ehh2w~cG$OTI@_eQm=Dh-jvBWPE#Zd@kn+q3d}Zx_>CH80 zM*nH$-V1n4+kehdfc9`zH$^*@b$IC{Lm2=<|AdSSbBtQ!ilBd~Dt~A1fO?)x8DtG(H1GCp-H$uwQp!`a3-o z1{sNAbylm-r%KC3PF+)0@d7NE{j&ohtv>h5=Ue&Cxu@Je*`8e$@IyqOoYD7N{g|Vi zvC;l_^$5^;v&ZUTu%%C|2LFxxznioGfFzHas)&&>At4@)PU)9t5uD(zrKPp&CW;cy zQ*~uEfJg@bj4vq}59)rG!c<6vait)w-*Fhii)m!AimjwAvMcT zz}oGZ;PYm89#JTiOR_p)0s`Jp+lPPoM!N(J503u*SI3XDnAg{Vl9Kc1YpI<Q3Mqt!${#fB^}Iw%e{_U-*WBD8+lITr$0 zwPBF{{(cVWzrM;P${v6GH|nf93PKLf)<#U)y(*l79(gsDV#=zM<_jQ?ENVkb&$^cO zPq7Z|!+h@xx2M}lw&}gJOO(e&p9jn@uC6YufXxyZAM9P7IT`8ce_hPzgr))|Ht+N_ zQ2)f0k9?!o_hj`sdF1DPvN~p)z#LeN{Ro}_JSQU23)-}il3DxZ?Cf)HZXY5dfD8$o zukrHoT$a61Nr-PXJtqV2?Mu zqf2TdhIXPlCoS>m&u0!km|873yXB(v_8)r+z211~0!H`OL6Bb9hlg2}p$_F06vto) z!iJut2tYXCI0zg`SV|ylNZDx+=!b1R8Rb`U>HqsTPcc`#G&D7xR55EUL3fHHOhuGO@hi&GAk=s4T8bEJPIqu=@H?NWZ#T)_bA27rz(cls!fW zoUeg*?{%}wuH+Sx+wF0PdP(5pYtEkAvzEHa?P?%>y7WCW=9D5?Sm_{tuh8a7-HhKC z@*xzP+h``}7vGl}*X>pi!siU9}U2r`<(9<)!PAzqM zd@4V_2=alJMPY6U#Jr@d`HhODWk7&c(b|65F&uEKM?>@Oi_f!*N$EO0IEwAFuArq=)px!R z_cy^m!^+Br_=N_g{7$&?S);B!3_jbka(IeR{R<`xOxk@O(GAN#K7M@ckcj{MIsfBF zENR0*;3+&m&CA4wZi>zsK{6C5>q?-V-d8y}yR=4J$FxDzTY z@$Fk&G&I(SYgK%ecK%F=!RLdZd+KM>w0OCTo0Ad6Nb97v!^#^h1(*6&J(=(-C9{59 zTx9M$Zj=H{7@cEwr zPZQV{L0E^EdSn1W$4Em1Ox}72nT41?)^!DwX`Nq3j>f&$f4pJ?VnOm)Tw2+|=Ayz0q@L z0{J}iT}GId(A0G5mX5kyonEiLzC5*#SsQUv>ijr;{N$b5g9>;ro= z3yXBL@bN*<2TpoCA0JTgEs&7l7ZwW6&m;c%lU60=|2dpjBnQaP_go{DXOfN|GM}*f z!Z4O+deZFpnk)_jqT4Q*l-2yI31f8h%(>2ow_ZiP`R}zcU))S>o6>bD3V>8weg#i( zyI9zOQ*EbYXQ2Kv$ic?@&1)lfm5-XH6*WFbwEFqO!TjE$A01zt&Z0cYuk_&()+2moWnU|+$b+46CBJLk943qE-53=oigM-P) z?{2S>skXUZNl8Kbw$;`5gRYQ;T-$t}J>w^&%r4RPd)6WIIJu_@v!)UsocP8NL8P3@ zpb)F5#jr4;0rwS1paYzo6pW1gf*JHxb!O>8o@QV$sUKM9-OUkeCCV z@SEZPSa;#*7w9+in42E~&osj;!YnG;@UaW5PXNLQygw&Ub(T8b!K|w(o7~r$x3e=bgOF`=m4INar)EeUxFctHm&<+DL`oDJE#&a$H z-iHY9AtU|Vs%CCa5&ogFAd|8=x|_p^k$E>YIk2&Dv#2NnS=me#i;Z(4OQCbQl z|22!C;Jmnid3Tw8GW2f(S0k^oJPKoETUQ~|Al&rSaB8OkVkdH0+z~z!whhoJd8(?~ zYGwP${QT@MR8X;cF@3jcnF={qM>Xw`HaS;7tPPxIrTTO&9)!yA7+nQ9QID9(Qc#HJ z(#v~%Z1~9Dt@~}Wath?RECi_I^j^GxgOKBC{OQ#(O0@rw6A7`{q{02O-Jtum8{;go zqMXEgn~#aRjIS|2xg-4I&iGjL=Zp4%U%!qjkE)&HD!uAvE9*jXC+4)D-8WSvq_DsF z3o!~F7x4Tg@q+M1`eXj7+JD8a2fPDAsJyy0FfrA5p6X+tUeR#DYjr`LIRehE9YPAB z*1qS*I#lnq;HhShw`vIa)(s?syElb+NwfV{Rx3v4u2T>fT1DPn?z!Iiq4= z9atxWnaju5kwjT04+8~#Y?Xb=UTl`(Vxe5(4=)jMOrYvX6mKZ1v*7Cw?9JO4*?xm& z=;Gt{Qn)4WslA~k#Qmt|vg&;aN%)r6eeJRrMK#^CXFZdeuT(MNv=yHBg2eoFnwO2f zu(ZvOxq)kt3FVuQ_sJXNZykzvxkJ({4G3KV1&myo9qOdrz?<&1Y{y+QxWI z8ksYx@E3*l4hJ`Pa-n|{CkZE}(HdY)ZvnoLj3ei?v9ua2AeVPp5Vr~vHoBfD1WEve z-iO`H6p`|!M0j$xhosp7sQI(?eme6G(TpiNbxgr$C)5eA1Tda7HpBLysRX73>yMT+ zY$GE$&CMY6Fa0186!auatKBk(8$}hy_FyM~Knr0o)*vT8KT(xUl~V0a0m0RLKShXR z7?ns52lrrb&+Uof)!VaG-aAB5(W^J7Kke!mm3)bbmkzt1lHm_c8ArNqPs9ZUfSoBQ za&HSLMw3J&Ez{P1wq`-?Y^Sp61*yeEy~a;NPG!~Bzy4Ib4RAxOT#2Y$-8~WGvtOT2 z+*mp9f|oKS_(KSl#CtxwIl3(q@k-m2N~JW1jBHp%h5e?IM0{y!Kwe3BZV`99^@zw# z0wzz-W6!4mG-|+x9{`u~@{y^PxBk+pWaX5quu%lfzhE`oAG{5F{4X2jyc0mE`_4#G zLX3}x&-w**Bjgh4!)`-r_cf9-LFZQ-1h>YT7OoDiXB|u}cCq4pl+bfd!aTA?mcWVr z?{rX@A)xY33UbSB1cZts1$FJr@bPbM@r&UEcm*!D>q@-{rSgf@PGK4GH7JUV_Y;`? z{tLWw(9_SfxR!HpSbdd{ki@*XfnSQKnTzE~n~IA;qD7C#nv z>+n^JLIlc<@bpKD!M33`@}FFwE0`I>QXOFDN&<057#9=0yrSx~Km`9JNI4dAH9{21 z4QQ6&BZ0fBtD7ECuqw6b6lpcz3Tfls3*>DG!;P$_Wg309bGYco^1LWqJ>u)p+Jw4= zw1zyaR$txiq}22uj6#dqi8qsjO5E9Jv=LAm{`)R&g6A(F6DdK}tzlc(HG7XjIMu+1oF6XGoFU(dC;r~D} zrIqVJN_sI_2xrYuD?yor{qna#lL5c~m)|JRlH7#wgp$uKEx;=fEGS^$92*-}er`;m zIY0qiM2Sf6C(Pz8pQ8h7I>?A^LoXKs6liWUfuaqdnuCMwzXO7HewVSTWe_nymrCnd_pOM=ox zyrgY>!TYv-#aMSt4z!2Jx;6=~#)=CIp9|PgC8@%;*25sc9sTm$z5?QV9~uhB!BJW9 z@=NqPPWtH2W8=Ic@)TrSUkA+F-b{5jyB&&(m>b3_VfMqvj7mx7`5fU1*~d+#D)xsG zvOs%VhP}LbVUbODr}UYS0Ac5&xjCeYSN4U~CCrPhaOmOUEg4^-PVJZ}m6FH2{<$tK z;h(O(bn`S^RbQy1((U)qjih>$M>w!-q!JM}HZhT$kZ^;Gk$x+H7gY|Eg|tB~2g*h;B%~PfM;8bU zPX`tqDjp{vG;JK!(v>9@MO@v)AqhbA87mE)5fcXPRDGb(3!L2}Lk~JX$kd;V zZ6%C87w1fiqmHK{WiH@PlC&V-B2JrG*m(Y?W@R18ZS#$;l-`LN*QyHWlQpO`0#QZTxzB$x!?F z>gtx@4ouws=xp!)d`%P8cXPXcwjwxTV%`1s8zy?MJ|I^k@OsFrCBEL0QSueZO-5FR zF*0_%IdG1iAID9?OmX*<@#&~fqc)u`N;kbhX}Wq2mLCz`J18p0MYBTliHxv+-1E1$ zA3Hh%bJ)DoGXUM~Lzo2hbjQ@y5CVgMN1`F;LcfDaB+BHpbv)REdnw4p#bs|C?CR*=w5{F<4B&9EU?TNPjtr9sqQ=gt^ z;=7n&e0!Pi7v17_v;JYB4y6E5`+TXFGT}?@q)h(Kjs$Pr>kS4l12n;p(F{-g5-hp9 z^EE%0D&5($@Rv7<-7sHm=20geebQ)(y*@Gzq^aZAmtj;vG*RLbh@Uo6l-~8xxvR7N zu$tz*yZ3ynr;@(5w)($)4n0Hth`G8328;b~)A1qDjuJ=;HNppWLU-K;H}|e~b!}8$UVhC)&HvpnMZ$I0EnhCG0*mPz0o^n` zC?MjPDdOd^mVs<(eLc_Mh40WrP(db|LZb#k+83;Fw=yZM14FduT>p;K7r;Y&{?_j>r*91xJE=L!A*dQ9l5|1 zcF@`T{$~>kFjF;ATi7|`A_^21nE)C2MMrriP%n~;8 z4omKtqfyE%^M#x*FYe_4ko$q?;Q{PevXnld5#3}JBGJCv(qYolgA#e^0%|c(ccrA@ z0W1M<0v>*?HO|tp_vlF+{mApsTy6ifiCLYHja_><{wI;;C%L|X z_N0jLK+?5hjABgC_wr5K@A;!MH^pVwKbx3tD;GOgLQ^4g)YO z6O4@`(>WG`8ANzJJWO^LyOyOcN^C-31sV%`PDH*UU~NS?C%G9Oy}7#9*3l>7JkUyc zx45{f`a^0WJhJ8u(bmfBY`-<%l`$P4BK{t0wKL+YELQjDKz0hqo~>r(KJXIhHI2$if@1H)^^k zGk9LAF;`_NnNaGFF^Vk77~B_F(%jvZkBsC|-yBb=Gxaq6XVA?LI&a`905qQ(`k7rW zl%kxXCxb8B8~2Ct17$ocn5D@mXsVt3=)BS7U#0N?n&J4ma}rh+Gh{p1+GCY^vF(oyseWv$M4j5l2;3MR7mEhOn53j+xTE zP$iq>p+2&5axd3jJ~c1WK_X7BdvpwLxYEQ`8@^QPklEnePERVm`uc>)(mKPjUu50qZjXf zmP7QjYZ1`Y0Q0lMTIOEm^oWwqPD$YO87cRJk()U3%W?LS$Fmiw1~k=Y7~}`+5RpZh zxTg7ez-Jfwloiw1nB3l;(dMTsFE1C=X-{&K)%KUahuG&5v;<&+oSB(9@+xGUSS?Rw zCD8?>x0@xaJY(ajCga+|YD^DQqrY7)J07MLmc@J%E3SdZ2POcRgh;eh1y}n+wW{PIE;Ty^!-~*(&Q{9gLb0lJA zmC<49?ChMc##;E5$AnZffROBh)>g~^``D^bTBC6+{(}xUN9_QQg@qDYJ$-@s+hX6Y z#(^xNAc?5RsTeMK;ZxqwkVch4gbylQ13_-Z9sFuEMk|<3cLQ$030a(TlG3{H$8;Hw z3CJt=F0Gs|pqu?S_6@V7fe#KY^(@R9!V+{1<;d@C0w#Iw?d?GS01ya1z8MhnM@B|K zXzKDli8bHRnX$F)q!em2Ywc0iR3d9LhkDriHg@Nl!81-O>5da8 z4p>~PS*t6)^^74tIIhxO5C9J?iNwmC!;DI7_BDFkB9~;d=pMm& zzTGIz>q=3JBN}{a4(%IOnvYvv{Ua^_dHFVZe{(wzke4n$9gr%OUu^;hS8>LO+fWX{ zgycbN_oxI~Dv6xla17O=4sl=xK|@0WfrgwKP;YXs#S1^t( zvAG-lV1(ETeJ*H=a%#xfJlX;*ilB02%PT8BE^=D=jS5!vHE&UvHvkBLOIldCXVJhw zM+fW=Q_z~0Q5u>bVPj$|Zz{E!U|_TbP=oy z0neX52Rm{oGQHg0r)Fn40)FlHma3u;)wa17Cy=}ray^}z3Z)^YdPJsKX`k|FNfswU z7p4=?ahH@&7w)8)<00iGXfY^P#}~}Y{p&}w%xe-JP|&|EQ)!~DOQcdt@%^M&zRKU& z7~l%8r>DX6z9=R3_Ag{^u@BLd91Eoa87x`>Gl_RF7{ULPuJ%rMal)dUzdnHPzeEeQ z^{t@>le_1DE6uBp%y>dUk+k;b)huq7NveVlZ|F!_wRW&J?-&QGDVhtaNVO|UaaRqu zD19&R?KLx|kZ1%u<4UuQ43(AD)yJn0Y9g6^NkgYzh90gvxVqLPvP46seVwBo+~mS zK6~|iY9aKS5YD^E5%ee~%d->2oC@IVgn=t8%-c&)_YELy_Djab(J@i;x(4@?f9$$s zNgtQs4BQ8RiN*Kt-+_z}ltgcm!6N6CJ+KC;J_B@}GeKLNDTuHlMiB8}fB8QV^S~5~ zMWf_Htf_dUKPn|R?VB}{j_sa{_)eP`$4SxSs<;O)bCPtHvk9{djrPmwL7||}^0u)7 zh6wR3K3WzQ7agnzymCpMq?bfzSSX0tszAJhfujl}PM|Ea$(LAe@fY0Y5%R%SZM-?I z%$u0tB*9#vErcIIK3XI{eNKGHS9HH)VegTs#_>5o@>7^QEw4%QdzFN8*E1Hv%SV%9 zUxtPMsHXt<29^N8H890|f{j1Knp2ABs{btIkF zscx&)hb=}>EiC^U)=-O_TH9AD{}a}i)D@@tUXTwBZ~aW4y~DpDW=!R}atWE1&|lr~ z{`Nz0PA_g?5;_)II=$Udh3e7Q~2X7r?Rwa+@SB8ajA`m_WZEXCY(Dc~($yIS_VS@p}x{a1W#V~Z_R((Ca{k=m` zrr)#}u(t1Ib|>aOqh`BWv>t{YnJ#UUuXhF9ME?HY1Bx>CD*fHH1O}dVvL4q89N(-o zm?FiuP+dkl-*@v|s*Dfk&?cejj}NsL$v zyBalqO}CnSu>2|=DQj|g-z3ZeNZD&stz_uyB9}Wq2Wq3EQ3!FrIr-nX`-5h&*vrLZ z_`zr$$=>3;nQ@A!$dU&55xFKO1B>;o!d}hyl;p_BNa^n~;7Rd2-xk%BW2t6lWNbGv z1qBA4o}B@27gk|cFSiW|6%yp^R^EoQ2H|_vo)`q=H2falPL7kWwGZS+&a3pM#uez@W z(Gx(+N2yFCTgI<6xm4g*j3_Uwonx~>XY*L_}b{1Yp>dx^3w0 zlit9^aNRe3uKq@dP7#mAWNGR4WXT1fsG`7F%tZ?o6 z_AVKCt>r8eP%n(Smk79*Dl*oT^f4(=Pvfw2$KSqQtS0_V>hqn?&_>3bQSNbE3X(`f zxO=)hdvG*t&Vz&s8L{iX5#ODFkfBR{2ZYaz^qgPusl#y0di)NF8^~9txRraomeUUZ zkEE-Ns)Fm9gn)p6v^2=2rIcF$Q_Jn!cZ^@mFq z+;h&}duH}b&Eletu$>*a=3vAC**&7Vy4-X{DL?3UtX~aZd-fR9?3lkm)CqvB5&W?H zr8?rOk7MtxaN?5S);Bk&`B(PYXWJ_BF&2B+;JkxCIfkgn1053p!wh_2e!v9CAX4=~ zHt)))F3>?Fb60uyLn7>p?%8ab;;qkij3>!G=allor}J+k736xH@zTZpA+$9E5%h}B2c8Y(C+!T0+)eN%9{$DZTI z>2t;PYy9DfyauVFOmKwVRiMXr&w)|_QzFv}IfMe)U5UlDXpg*1>#uaqJ>=d+z6!d*d|_JcGGoIJabs=2OfbDO})`p_=Ae5Zt5WASxV!kKIsYFOa$lq8?YDSl(SQ`+!^8F?8ZGgRRE!# zUzlHJ-s&82CL|!$QPyq!CU`MOv$Jt(2+a{YtgWqe-^(~TvB%r6G%>+@n9;vvZRO}b z;CCCt-st6A;;)BiuWPGLyXv%(Tk(mhsDNH`H-)|v6#1TMIT@j&>id7j(`{uhl>yzQxE>2$eB<`lO)m!81n!>Q1#_xgT0``?*GQglx7 z&0jg0ARC0U!LF9vY@pE5vyL?yGbyy6rOnR~@+@Os?DsfL)xm*Uo%e7Ondr>ai`W9B zUNk@0%T3$lKz!-~kPZPKW<50ma)K{kOkNy8p=xT$`u)^Ex1=>3@Rb{Zw=DPwiQ^rr z9x_ww^n8BO{*%YA7C%>K&>;!ED>gH7pT6VL#5MtM79?)vV=QYwJm5~iFjrRDYJxWt z5qSNXm6wJ_e0bC-2Se(u_R-QqVotp|rc4ZY$2hb=1Uh(1Q>)&q4T=KjW^{DqZ%IW( zMRGTlC&hMnFIiIX!V8L;KILN|pC5$`VP5mK5&jWz(PUoPB~^$4!Yw z62!`%o5e(gUs}(?y@AF?f8~;7D@!E(CT4Vv(J00NSb%EQoG_u_xIg#VyN3R8=!POV zb4t=hS`rfvVGd-QbVi217?L}O%S{)#JiVH=S| zOPplF)@-Fk%xK|aF7FGvTX5;5@iAE>dPL5R%iH_k%zQWahyIidP_#eb2Z#^grqW78 z*be&}Awe8(4^dH3F_jBV^np(#-^6GocIUTLvh8fJLdBg^@l^)%0Kx^-9sfTK$HV|; zc(pxzx%0NrfuiAM;c@`~ezJ4<=Ai@X7eEkeYij6c%DDxfA8HI7KU=e(7x!71 z34gr!6F#<6Qb}gx@new(qs}=wcURx$VYlQeIm@(M_rv8W@a_bZRT|iuC%{EK@n6@9 zXZ-|Jw?G zE2^)kfJ?b(cD31Kfs`geY7FSb0h?@Cx-`$(MbKQ8;J zBblXuJ!SpF?>@a>bG)Kw#wS7XB!3sP0Oj@jjKRzVM8MOwvkSb zpoyznx8`4IfMGf*D=Wd`VV@EOK$vIMhY!s}uf_!*i+aflir5$0Mn?4|1+>Gd=%6H-8pmL>Blg4$Ka{E$F;+kBj+X?1i$?A{QOSV+4l5*=lN(NH%V_~9TUeV zq7c%cF!xUFLq=wDnhbz5)s*AoLC>MWF*)Yo;wUXA9jNWKrgE0j7~JxT_(!=Z*KtH! zCBsGEzdM8k#ukA6FfpMQpaF$KeNBzZxZ?fqTdeoJE?RXdj2#@Ae6#kHDDq#!<@%?&p!oS4qRv_~U}@F^5&VROfy%to zh@2Okm{69^qvHnlvYoO{I$2r>Bse7G?&gMvj}NWuUq_WG?iv1y^hnzZamlp4-AEcg zID---2VRjwdz>tFLe(B;c>Jvo+0*kU_B`hP>z^D-|5pPwkER{iaWIbaG*STSP~Axr zT>l_8xBsN-G4xjOT{xW)GmuSejZ}d7H7R*!0?e<~k+I=9oSbnM%SQ^v$1xE{v>ll? zhQi%T0fmadwD2D|B`RxpXg#1D0@Dp8QNz#$BplkJm^jwYEZwJKyvyLFQ3P}JTgDto z{lIY7rsn2_g#`-^El?4ioiCn=_^qp@X%-uCtO7mU$UlM1=UfoUmD?il{%zNR9y^x( zIOg;;P{+&-Vi#1h&5aNSLOId{BLd-MzkqysVC#i~0ubyCwq3VFt;0yxtJnbGKinNj zfWfS#;%I4CFx9waJ(Ot~8Q@sbi7bgNIS}_{yHi5awv~)uJa1^jvS5I_wA?P^1@#;W zH&+I~It9_(*jNA?47iYrohL%Ru4|25NbzNm$7xoup+Mro55WEAT;vAn4{0PMe`TNy z{eN~&%rwOQ2W5T>*Dz!0Y}0%m716QE(dal;&wa*f|2WElhIflYOpNx_Gdw)pF}tJi zm;YaIKZDV~^S0!7P6!8AiL zc9C=7xY}07#X!|LJx`p@p*Kj)!2|M|EzLjwyCXoCGd9-wN-G;!WcU9g_M96W?Z*20 z3qvJkWn6=*r-Eeg*Vokdw-DZM9+mvpjy7kdTiFYx410o!1dDuDVu{qwW?h{)qu8#- z*uO}F-HO-m-Kjz_6&oyB#tHOVrgm zt**D^6dtLeOVfb;dre{C!LN4;>gt>{G{9;C!_*XPaqF{|cLgF~?&ga_jSZE|yC91n z1KCiA^Lm2B!lt zCeEa78v2{;P56ivohR#;kAc^indSI`Uw8NRKzLsei=&3;@+b0dVs-P?RAn$Ov5Qxt z31PJWrr~7hpBp=k5qYZ?QEvCqmPMNqo9Q(shnu~Aq{9gX*%6Z>QmdS#}XbnUL1rc#R+ zw5;c0W`p$g25JizGYc48^VB4kT|ek zAEuZV{a9-sa=6YcMF)-t7xlf+%c2q<-`11>(TJ$2gG29r`3wUSw~1*0h{qw1uo8cR z`k9JH^6SOsxe_jhRPGZy@wW$Rfvl71OnYsKUA6%WqibA5VU5qGe!?84z-;d0(^IOY z<@s%F8m^^FaMOo8I;Y_|FbhUHh7 z7+7wO-^p5K)%3B*Un~Rnvn`6Xxh0UY>Fw>kiuVS@y#8SF+lRDH9}jjLe0a`tGyBqo zJ}_3`D_K$hG#j}o^xUGi=JbNOy6c|gf%9EOf+Joj{SEQTpP-cU#yCfL-r=&!O6(X& zs?tYHsfacy7@l}Enc_!87(Ex30w#f2fv??q3Q3- z9?v9z5?rUJ!8Qxnq8V^>Y|_#Ki3mVmXi~+GJtM|mb`uOm4uS`ljXbo!rltls<8r(Q z=Dj9{hBvOhS$?+jb)|m&I2}@$6x3;i%mYNRkUruHQ+2N1Bm=ojGm>22QmF=v z;BalZpVmH2q+h=OgbR2!+(oM}5zpml^RzkB_m9LN-`3jj4K6p>)+I_N=}UbL{jVu6 z*Ajm|`|C)N1&oIp&em>Zg#=;F+kwPHo9td&gJA<=#3blJIENjH{z~D%saT}`CbYt_ zOWRAaqV;3+=HaikP@XoFZ6sR>9{W8+afFR|5AMpySXu>@Hh86=O znl8(Gd~uQV^+nF%xgR4Xk}Q$+mOn9N{rcp@MbWnEC|g@eN9F zFDr)Z_Q>2hmt=1PiBNQnQ|K+S6RqlAR?jfS@JheWUg1~F6L z@h#rBL44uKw;r*|9AxGOgu+3~S^(h(GeRoc?%JB6u`wbWLuuiSa4bm)eUX_|thqUM zgKhkV`iPbfCWWm%|3l6mN;#3w@6ZlWiXRm2J^zd^?tF$27Y^F+@<@^1d8^-xiov|1 zggj-Asb^Pl+Nn>du88T!4o|$aDB7&=P(2^ZrbEqp$fmRMqB|nM|9Ta0hr$yhWOX~I zHDbp`dGUn&>Jai^W!b(tOC7M1`ARv2I)rvRuR-m7-$MFp(4ZB8QSc^+C4u18q6biL zg1oMP+DGzkgD$e4JF}J}A;MG6t6_5WJ)PtiRT>zVmxi64;36yeJ(p6_6cqllwzADk zg^dpx2sBK{?R8pcv1#h*VX}ow2Z@P^y+B58YHSSn%MJE3CMOe;k_x@Ab$hmkI3o!O znY9vxWFXK4Di|p~?nasB#M88PZA3)v?kfX=7s}Ts+WVoOmf-kpWaeQ?)0a1!m<_h`Yro5__@^4K3H> zyfZKDn}G|s=M-{RxEFm94CnRo)(rbW=#D~E0H$lEB&{#LG={`SHybmz8{VV zmvmQG&^Jgv^4ujIeRPj*!@hTYIwDh9@)J2kU5lRm4l&t88J8~*?jTg;o)&{ap33! z&##*rC?GkukJ}>Wnm8>s2l6NGw>Y46v*da0=Q<;Wa#Mt?1bx5#pJ~}0X;V=k3>=gU z9HM}5QC0u1!_a@pO#R>Iyg;a-ztEF;`GU*-WR2X~&g!p}Om%%NQaQxyz7Ck4B&DVT zX*aM(tI@?RMK2{yfA`@-HHGiZ+MhUTzvo9c0D?GExdWBN6G(B;Y4g54n5zb_DzL7l zrzDW)yI=x_O(Pln*Hh~HJ|KJvFw^#XW6W5GYaqhjKza1E0hk?s|6pdj(uQ#eAumFl z32$$IzTBTtNat|?xi-M|whN#ER4wv)m_6O-GvptA2^?n#LY{&1`tHt7M>)Ue#m-lB!d303yM0y^l+ZarBxRLr%Dy}X zMX)A>t~g`={PxX8gm5v~xkXBmg+c=mU!n{E_ucVv90MhJ@zbac4g(yXL-uV^kysKP zEY@>D1lHwA-kUJOUKj1dw%=jc0V@smH>X!{UsHgZX+EvB;KfT(suoX=>c z3qhG^i~6f)z=s)la>mBS#>eAc48qZZYf`S;TwPjvaE*j>VOkhJ!k7^FI_d! zduvnsuN#0`iID{cR1?-HjhbHBV@55%t4*(_vw z9W~0OakZZIk^m5?#S7BtvafdY`cb#en#i8|Vs)(@DD_dW$b*seJT~z~_-~FQ@R?3N zYt*oKKaI<<+5p$Y;-XBgTFa^Dr)%NWx8&^|Rb^#INzB@rK38+#d|igqWom$&iBwK2 z--pva;Pq->3(SHS>h1jQc8fsd7DlAaY{gn{G|A}l*^?aXWFJlqi$bWy^-xQL{OTFZ zTq_L@)Rg9;qKNO(6B93oxK}#%Sle#at;$=Gg_UJx4=Ve}5m(a_mDzM%FfMqwe*b#O zcP<)}v)USDfaAtOUoQ?ERORI{A4EDF{h{34-2PC2kMpXEzk`n*`TqQ8Yhc74@%K&2 z-<<1O%og(}h8ARN#3=6{qda7yDyU)T>-fY3&{&L+@Yq$*LC+|(_a?3iEi|lk9>gkNYqasxR{yMe|LxP5 zieg-ohq9tH+CxoVhOWHYGz7~NI;1Oy8E-C+K2R5$IoZLi21G%QCt%hO4Gq1&z0G-X z1m;S>9)l3KO4Y{S{nX?yj~Digo(CeX0^6va9kTPNQ;J=1q9Av&5oO4sl1yo{U@Co- zG2hVx!{Kd4SJx*pO9Wo1(rA(*-#f=N9z@X(jr^-&*XPEPIdiAyr-}d~L-tQdQd3o_ zsjia;cu+OlrJ({i64~X|W+PqG*+4Ui-t2lv+_JTl*Xgo9-T8iW^y7V<-HOm^D+nZ- z^%LRD?%dfi-?8h^rlTt&<9CItQ&mxUS>^>^4B(bAOZoY2M?PRibl5<1Q}J)XuAK}x zOY4KV6~s}=Yk*}4E2i;@t}^JDGNHDo`D6W6MkYNSIX_nL1%CI2yj1a8534b&s4XTW6GOKDgV*3tEhI12Fb`E`05SncFCzr0B@B z&nwrq>{!Iiw2XV;PbgGnkeY?Iu(dRLHXd02m^-Gtpa`*LPZ&d!OM3@0DX4WB@X(w~ zfq~)iV&nPsieK8U-eLc9qCI7cot0EF(_pwiKZ36vPAfnXZr8F7x6Z;%awgc?0B_o!n9);Bl52CA&7i%`QwQ@tju@wIC?MgFV< z2B1~YI5nEvg?lhD+h z8%pEW*3iiAszXIZCE>Oe@V+^5#(Fx4;1~GT{evJ?l#0g?2a6t20Epfs*uo`ffusru zSpe~yz@f_8Fd)3<_$8NAV10R|rQF3Jd5#Y2$3t&Wor`Vg^&W*-Tv_BpY&c4~=k)Pg z8h&C@ma~tmVRWhF&^tk!`hBwiAL7jgb790?;GRM|Qs8a9U-u+$thc+C zgW1C!11aU7iuy`y#w|r{Wl@524W7G^IB3J)(LchebAg3&xl-!?F2O?!k~zRiOb)lP z@dKS1OT3A4p|>Wy+G~7bf*qbuZlR>n>EYQE%fsp2;Td25M|Fj(f}}doUF6)OyLA@4 zu3As1Xq7B2H@lD}yIlqQ*&BQCKY$?w$WH?<5WU;mp(rTb+tk$X(zQ^Hj9$1i;MzXrDApGe3aO2=#<8yxnuY^_>x!rSmapApNknArG zC?fC%!EM96)YZ{h4Ro-#cfXuc2l;AX218XJ%kq0_w4SRX1gG!Z%ky?KIlv%?>gzE+ z1S%{nJbQcG{!G3@ykcW!w!IB%DhSE0@CS^x2!l66&ETjQ$lvLb|F7geVa|XVkFV>^ zj>W1g8w$^-3~s8F7bEXkWOH8#0R1x9OwkyO%zZV``N7rRF`R-6*Rgs-J(59krkM;! za+5o3tNXcTQ%LeN`AjXjK84ECb4!mIeeR`Lm=P3;`b4kH_7{_vl6M>QyDy<&=S6Fe zK7;sd$Nm`Hk>9<&#M&A@KQ@G1VBRz#htoWNfWPmX%}dO4WJwQ~R%+p`i_Uhq^Q)dp z-85v+9Npi>8jo-Iuu>+fA%&>gIrHI`I;dNs(0Z=){K5?*C*9+ z)^LCGb~VsyGjqlU$Oj3#BnCVs+N683C+lpp{oYvNj9wWM-G>B7il+EychsNgp0Lhx zAF*Lw1x#}z{Av#2E6Ac1gG~>!%v$3jBQkBR?J{T#b}#i5{Kz&~^mfbNYNZW?I_KqR z8h)Cf?cC4nXI{PR?&<<+s%+MR!on?c*LdXLr$d%HQ7CrL*K z1woK-pTuC!e2)W(On;ME!F#h$V%Gw{in3^KV}mh+qw@*;dt5?7(5heiVUI?WaCdcg zuQi~;AZ%0UtSxY)=m}xRT%!zNcSaOWcea(&YTA0C?hyFG1nCC011#`^fbDhZ{PJ`O zQqumXHqo)t2|y8;y=P=(q^EybK|0*EG5jTbYU06mksTQXVfh{!-}VH}q7 z-ah&MOE1y!)p7%M%NhCXlf95tgO@!=273!#;r5}sE4kCl2eu4;;z@yj%N#H9(Cn-{ zC5KHfUP)q+WivHTxCrHTM1%>PFk7c@)xH zQ*UNs!ZSP94~6!h;kE#)29M@u5V1GB9xu0(e7w@`_cw_dOoK0N6~!^|d_M3(5qjZJ zGKKxfc~G9F78f6e`HyAeAZzpPk*pezCu`ks5kLxh(kkF6plnkCSuF@3YQw`Y?+{9fj=v2xsBXPO(PzJP|9*^pb05;# z2WMB((sujV<(Z!k<@Sxo1@GUH>1HypM#K0uyNXmsF_YwJ|6WXv6z>#cxkNHhk2QzX zXAw267xnK!#S^Kk3)um*qcT}e*aP%QTCxA3aEIJ?cM6gFLf^4R6OpdbB!j>K0xKTk zQlB8G^#qpwt6ze8iL)Tz)6-vJl%=+fkw11Ycs5H0Y2K;6LFui*B9mQ*(l7i^*$tXS6$))MjRn{xxM}5OAPx0r!?AE{l1YOQ2ZwCD?>4f@AyX;a z*BEk7p(IRqfy|NS=?=YQ{#Uj^ei0!y@w^zpIpw9D9Q(Lxg~-bYq_BvHz$aWDFFTam z<*t%o#5ErhLCNB__t8=28O%c@FfliHAx%scU%dI$q@=A4iiWOghxu&$-`>H(AN(-z z)80~D=yQCacvKgVx@kB6(l)w%;+oh2@vHf|88AEeQ2W5DMZ_^&U;cD`@g)Ah@K^y! z!lO)Sd>h8gEO}m@abZ=@lzkZkZ9+teUznL0G60@Mjm?b=fQeje0*X*oHCG4c!GS^V z*Qm<|6#koPzck>(US_zMcV>?mn3{gY$�cNlQzgfUr)q(w*(?>n?Owa3TRd<90ZY zYXjES*6C^zxpS&4)mzawX$m2_gNJP- zG5pijs~)&WhB!2z3UO0oP|5&{fI=NOc8 zLanj(P2Fb=P2`^1+muOtQ_nld$)e8_0u@VF=x{A&zl8jjPAEUqct3$?8#89%?z6T# zV|+1kH=$sE`>)sBOx`h+l)*D^UYud$=~45D1r)!O?D2_#qtA`Dc6`{=EE!qrzO>bE zH?+48y(HXO5dJI-7luxUr ziL)QZK24v5N9lRvcZkCY-z>Hn!m)2pLaVF{RPw6)P6ZX7<%qY~b{5Cl(Z2S*F=QSo zgg0j!Zw_(osi+sP=n#@Wg6{qn_^dQ@^3wvjq3rTkazetzW441K zj$a^kN;g@Pq`Ag;XJ_Y#T!9_v#hhXsjpdM!+M!YDIeaOIes5xN6?o_%Vz3`Gk)d@! zV*1gHo0nH}irW3V-9nAlMxUrAXg8~9bXtGBc72 zM?y!>PF&{BFHhAm`40IJjj4%`euP)886A3FBWeEH9=u79x~}Dxyy}fnUP6)QMjSE^ zZR0JdagZd*(7&ea-&g(VeSIwl2HuFA;a1LTX%|6^s5Y@Kp3q*sT5NVZ2G9d08K3j% zy10L92ESV_l~6izj&cBj#6_6YcwgUkqPpH3fI>F^Ci~rP5_U*AIGk2OPWoxE7v*$TtM`Pwut8$P!qQV_854TJ>G#)E;l_5P?Cgj2&36OnWt z4ys4|g*^ZQCl?gV^t!>VFNVxNrlONcy{rSX19?q58!04GSzn*R)qU!x>d6`ifnD1N zv-DFRh2N_8{Wts-%~$ys&cE>=;vEFh1EDQ!JklMA_fZIXXXobu5uJ?yVD1?6??zxj z#;gVKU1};Zm(#eWW9#jDq-*(Z$8Fu`E7lV8+1CkSQis%!!1AN1|Z& z-sYl@!$G3TwDQ`!eZE)Nh+4?{vHP9LE7+Zm<{bl!cbr&9;fLP|ph=fe-rom4{uRK9 ziuULVEfA#6DzFc}vUGdJA23oP(ijzhZF11rJVH3}raP^BcH(T`N`S<6c?|E>H_9Co z7>>)MSw7RiZp2TSyltM<SUVjlIxe~EA2<{{5)pBoW!CXEk8xgOhcOGnyIwKTe_EjA@QMVAh zImuRC>R6>O59WL&|6;{jwr})=p7n>vYeAFA(xcER5-&-4q98fxaelz7*7;rvC(~fMTGcq5|NH{SPo{%E`()J3IeV!5`u6 zw|3+`|1rbva7?WXB3buuG1g^Yu^zSu*TY4K_^^KWi=aFq%LtmavNk_u^NfCdVUrQ` z@}pat_e0WAr=J?f{R_R7CCaxFey?oD2&3oVqo}wQ+xR6C-4rlG(BXV7LbE77(7)uX zfk+I00L0)YUCZCemR9+B z`SWInqf;cW3$YMp*8U0;_PDP39gT6^C|w8}!)OlTf+nb-8mC{d>t0dr1pFq?Y4v1I z!f5W?P{^^~A;GViUsjwc?BtvnT#bm!M*{|qh53~%`sVgHehG-Jq#QiBBqZo$wVx?U zdF5ctjcz-abAeB3E&5-6bms7S?7S%Vn_Lm6rUNBvd|b(%4-6FQ-b((dh0A!)dSPFi z{7X=6$;M0e6b?vjw_?bl-CcTWFW5Ku+5WsxQ0XE@wI&+eCEq&P9(B2lwZ>>CvEsvr zO&0G-W`~{Cs5(iEH-~b6eUHmu*>e}-(KxvFZ90c>_@!40;T%^@YDBbUms{uD&y6Vm z*ztZEm#Uz%cSH(sJ)j%iGzS{9c6-&19yn$yOWa---mY?^_kWVp(-#4r4EhK|{2Dgt zXs4$K50w}k)H=9>7hYfU-o^SMy-k0;=r@Znpg`U5Y>(}G(utz)_viu=?D&4#=O8Eu zo&BK(0ODV3(mIQh;3a1+nQMuF+R<%X!9u)Xs6Z{rA}|zB1nIEu`Bv`kJjCfQ-T`48 zpvor!WviB;8h7`^RG?5{p#E$euaXHFe>jd7RcaF^hL{m1>5!BJGi=HLL>893g2Arch(9`b^2Ov5gHA9z+9 zHN96XG`TD2LTuC`Ij0p5MW1CJ%m!ZzUFR7OFBdzU%+X&p86?4HL6~@`SMpad0(}G$ z3RFT|2x+OHCyU9BmF&=y(;}qzJ7K3zy+;vWBx^AI|DsT|Cq-Pc+2dlR`l>>ok7|!< zm57@HF>nL66B2f;U{dM=MNFU)k-ckx$Hm13nNnR3E{)4sk0$p)gXm78?P4ud0Ge4h(u5O?UHE7f*fYwL_GP(1T#ulk4h7SC8w6T#_ zYR7Zw3F~m*52~d4z3GkhX!$D5F&PI;OkY+QwlSm6QX8rL@UBX8jA|)5q5A#%C3~IY z|8`U`1c2GvF$YH^@ z-Jnc>8gX)AkEm2U`3KI&_Ed+qzlEO}qcQ(ElX++)y<1%e{9t+=sm7~#UcLw{9T`@Hq4TCC297`?$PEvnVy2T!Y-?^(m~AK-PE)|;5pq9 zVe5m(naPh0z|cy@k_&zrOrgtkH@Z5QJ85NULEHNT=vmg5lP+T%`$ok)3Aq$DLvS>8 zcS;V6a*EjtIhy|TDRAb~4U=n`ER0yT00pl<}=Vj@*1-8gc~l|DoveM>2iFo(7~UUi)7ljn*~+)FelN7 z{gq3WG+eN~xw5!6%~XGc0zF%9(BR|G|3uTaS8s(+#yEMewcVQSr@AsLYsrpy`>6Vz zTr^S$${auWF1(Ap=b_eW1_<<7XDO87;{SCumwblAv9kG~_oq1<7wdLtp*lxccgl+S zkyW~m?~MGs-Zdk0tF9#{{u0sm*yuBV2MHHRzqa3Xzwge59&+`bx@nyMEG_BE$sMkC z27v+qs66bg%iT|(t3(xt(|Mhwx^B<69sGn+;D1!%O(jt%xRUOi+-H51)Iy(-2qioP zM1yz~Xts=jK=kcfpo==;1mft`$gc|X^Rpl=l{iBvy8QZgIPT5Q>%)b5pOvHC?Yig> zseIEWd*4tB6YAVNE9d2jO!zmU3*Xy$4+M6ty@wYFHjPb8@<;K)6%yj(sb59?YICpb zorw38tCnEIv4$;u^-47Xx!ba^BslgD9~S(iF46J3J86p#8TOV>7(XQ9m0=7`U~x=y za-bV(nQl~_3_M}>Z0N|Ste2cy5$}r=5sBZFT88OgzL{B|#uvV3nvg3gZuzhiRLDdR zduUszgJQZdYLe=G*Poz_q3CH>@Ll>#_&2ltvt_ELvet}L!AD(Vr?b~5U(TmW_*8O* zt-n9Eg;Y1!mrwo_!`id^b78F&-w(J^u+y6m26$ee!T_t)nK65U^N{N9tL1^0n~P(r z{|Khu2>>$!R;;LLS=bkJ!or2h>gqHr=QXN@Tui}l&t3)VT8ch;a(Vd)=1rrd~zT%wQh079NvQ^MRjW`(8K|$RzJ>R7Wh(hD;KW~bLiwf=T9(Zq#Qc(bZ zx#y^;sO$-lJ5NkZ4Bm#NlB(zOiJ#RYhwh6zBMuUNpBS2#8O5)7MqLMZ_IDNL2OqMQ z>VywmGg=K?cE@Q1=)Y;@f0qznR~}WOxY3~(ES9vAb9A(kMpGThpN{_5GLdH(f577W z2Ld{ZsmV#-xe`t-;TA#hrO!pM#+<~bTbJ*N(cC4(b}F36c$3{n5{-*>zu8%a&;|5Z zeu-M7s!pY7jjblCbJOJ;rI>f6g&s!gUs`8o($%)W8?OH)8P1<7Ytai=KCqh-+|-*N zX}ekQKdXrdiS)Q!r%HyzOTh(ysQ$?d-;W=SS#SC5RU6nuF1VqVOR7$$>Ud8N3zhEO zot($=E->3^EP~39J6~Oz#+&fPHUJ(V7PMJ8wiz&3;Mv)FPvr7&Dw+Z8?5PkxG-A&) znW_F?M#M0s6>L)Kt!%w$^;f{*xEkSq3xzQ8J$cu5(KN@Hj30ercGZ&ZRr{yT9X8{8 zV5d*2JO0kY_twP(HqS6b`(1U8Z9V2Srt>%1f(d7=c!4a)PivuG-NY|V>mRzA^JmStaQ4K1 zw3pf&Mhg`)#or}NeS6vn#wOcq^U`0xN`4wes|d_On$eXMh&my0Bo?}?|C9{>pk-2m zlXjjE`>_A7026;tp*gW-OY^SAt6!v#U`tH=<9&lwzYL+YVTzA-_=|f<9zyCtV;uBD zrtivGM!P%u?6^(R{i6N~Gojxx^i($$n<$k^7Cyg=X$|;M_ z?=}yUC%c&e^weaEwovF%Olj_Ie`4SORly@uPCX(kh`g|4`sP2)|=N}d zk>!@wRzL(R;s+~@(+`|)k7NmYU0v)>R7it(fUUIzNz7v|p1N6+KP)h*O@9yJuU=EPL9vv{s`Aqfk& zcXWt2O65m~h$*?L8LB{r#v(?dU z9S>AiH1R-X6&?|hJJ$u)*k-IqT>e0ooGgdlTeV%Wk{s)u%@tI=>p%E0JU3(mi}(I( zZadYo7otgpk1Hti)S{NDIY*lPt8G{J_&rXGCRHNd;HKKpHN0tB-cNHj@rhR1vk&cO z2>sTn-|;c3MF)g0nU@H36P1^Ck|qR_W6Sa<-myuQKXnvc=pWJ8c_f#$=@rLoJ&q&o z5_ujq3L0f})6VfT6qg7j?r{yfTy?P>x@L~&%AzSZqY5z`9WdUQ80qM=wEO+aq8Omu zJKNjemy!N&Cjd}!fJp`z1F$OxNVK{6%gX=MwG$KkE_-;3N?`p@SNAdH?$l?S<0@TQ z|BI`C-R2hKteOU?j)lj|1xuMJYJw(7=F1q(EVX*U$XUvgq4+@$_nVs=5hI5sfbLF; zQ&Urek_?8SiSh9lq4`I@I2d81N*e4{9#?3>?Gk=|!+Q4KK~)<45>C;IB;iLrhCU!+QL^`)|hPN34HgFi+w;J;fO*r?C!1s!-<(6-&9( zjLK6n;1K=MW+U#mB~?hDj&!axaj*;D>GOCKE%D}T9n^?FSy5ZXZ6@?<|NNr4(Ee>? zL0k_;qFb3RGf`XY^T2+ZLCYhb#?dQ|5st{#9`G^r{~DN)cfeqlwtQx|wob=$bEU}& z&f$#Bl9g{WHH!$@;C~g%xZMZd7{9>coadPNztNmwX_ks%YA}R6r{q=)%?LxZm zTZHRs)8E6K&GiK80$b$ZXcVcy+Xmmog&$Vd^uL&gidifi1pB$n z6~mA|D2yTZf+-obQVx2(Wnw|lu&AeR*^snLPkbY3;%>>go-b=<>KTd zS}>*=II(093takQ-oE+-<6C#GlKd^9tD(4uiV(j479PABKN7~>{}7NVX>4zl@oJTn zq;U_^qM@XjxUFT0ZZ-jZUzu*R@6EnBKi!hzO*Areol1(fzr>&InY0t4*e!9H zDhIQ`7UzT}@B~6FEi8PD$QE+U-q#%JlNZ=|Bu2Zj+St5Y=b3?Vw+=|~)w9{^&t^I^ zc}!J1`$Ix}VTp8KXYZt!g~p}-y+pR~9;$W@9)y(5+D`|g7udNc3OSi|n9$W>=~U<1 z{i}jreNtj_R1@Fv@iUE8ur;BRPX-0Kx1YeLijtpSVO%KuAe@hCfSR7hGUW3B*n9o) zgT6|-LI3mT*N~IFsgghKph%5&$>4Z{b7=o(WrYG?=1juh3A_#E`t32Zuj!0JMAjJt zMU<#bY;9ShL-9zJ7Bgx$U4#OIDqU_V0^|5WpNym#pYI{`O~w!guUs)y`33DGf*G>9fGzGqQl9W4D#5Pc?c|o{3Rx_P0f=HuuJye2hPRnJMq~ zZMRT3R!iVvgQBBHDi_HQai1RSfcLMR`BL@|la<~q_$u|MBXThF1v;dHUo0$TU=A4= z!QxFQbf+m81coa8{Q(TU0xV~_HB`yE+7Z#E15Z`6oCN3!FB7NQ%*o8YC?$>GpoeuoS-yG4$ z#vjo?T!py|U;QOQ!k-c#GiA*$k*(Ub{cb%)$HZs})-IHkzcT3cNNZ|nq-15WYS3`1 ze?qd8@+bJRNTnNi!ljJR^^IGpJ8zcULR0D^rG=7OvBaM`va@4ZMb)Vm_W!OO_orz{ zoN=6&h5Xh3_Jn4$vnfGY^YXfP+8(wrH$MQcY;Tg_h{fd zwEZ=qA-#p`RWUMeDuj@Mq62U&8grqS7&HboG~Re9jE&cr9iG5uo+&R8JcATSL*{@d!H zyS!4i{`s?3^S*Udi}Dzei5yO?L;}wb)53xRMb?x^Le_D2G*#89k#}$ZJ$D>)Eo{dy zKpVx~g(|>9kt_4Tuc2yN@EC(k3+Z2G%})H#RM+0#T|>iNKEx1j^H8t+Z3(zYuOn$1 z3-D@|jU&HtqwR&wnjBVg2DF5V0{tzAT~t>m?OgQhG)Oke3ufgs@C&FkfC3iGov&Yu zpa%fPWDiI;thlTn#BNpWj#lSa&Zdj{egTsG)a4Y1QkL+o6%3fzC$s1}y08tm)s?Y) zaZpZw7^)D(oO|7z&*N?C`Y-&ftL)d~g`?-rVyT=PAz#Bt@$Zki@v=%gNXB4Mz`(%% zVD|sod+WC<*RE@Liz1y;(y>S>5ou7GMR!XH0wPk<2nf<3ut21d?oI)vQ&0(|8&SGJ z$!~J+`+1-5Pk4WL{b3(_AIN5LUFSJt%rVBSoU1AEEI`=5+z8k8W?oLv8NT804D~Up zu{*e`vHz6BGaDE0?c1n}izrD+zk1E^o0{0V>^~DmZP?Y_1Q(@A5>-j$na3ZCqNA!G z^1N25qGGT@m8#4sUe^PADu5)i^YZFmp1S1@Hf4q4)>SUQBZ~fTRePxEj(x>Fl z`v~o){ImwF5yrtmkE^Mx+;)2Q?Xf>P}ktqpv$FD;P!WRIVqvnBM6jOR5;$ zh&rn@R0rQSJa?ITTrJLB7-3*Y^Ra08WG(XZx>nhiYFNdWPj7>Z0-9U&WSPnvp47@k zZ*WZ~BUe^d7N3I{fPjD?f0qRMhzCxKufhEH-HQ1%XV~WBv4P(M)IN{ljmG5c8EFM* zXYA`w>jWaE<{dPO%bw3}El;$Fe(KlHrB6V+U&=(PsS9^-JnLcI@t5O6`&o*hz<*z< zrZjVPb+uK9V!D{;9$-%}x=wC1w|01zP{T#}1SKY~L;lvA645uZ$@?hO3-YS7tFmgCli+^6S?+}L=~#Io<5}v45VUWwB#yVc)n(oRpx4l%R_1% zaWuX_B_Y!2`IrPBip8LKBq+1qV;g~P2i(9@QX;a%VL}=1HfjYe5P+(H(>6ko{k|^f zQ)%j_z+;!N;gEDbt6_IoEf-_IHvRRd$Vlxxcb4Smh8|#>f(tJ25xE~VE`!JcdX9e^ zjtsGvCp7?ouVC5NF0IK&7fN%A(~Qp8M>iVKVZE5T<>us`dQ3^dQnyk3dB1FCR>E zcn9%+D^AI9r?NR%J0EJQ>=uPj^k0;GIM`W{TyplyR9B zh8`yIxq`B+=->b?ILZF}X=QC4VpswkOkhaDTf01+wEt@7aZvl^oj3cbzra-1LLPg> z=z&J5JXg}mvCD@qFSsmG4YiKZLph>A1c2Uu|L_o4!m_l0F$>6ZM9Vz28IyvoRTI<3vIct}cI-qp26kx(rZ62(4@AD9O8d?$ zI8?{~JyS(XN$HWak(QCXp5Em_QnfYSm!#xmLn9+m5fPIHM+Ok10__4YLm+K%aC3J* zmPxeKoHZ@VdIk1DPPJFbrBmlCS-wMV6HQlCMd#a zmLkGoJq#gGhaoR&Q`_DgYq>bF8>;$g>L4bh%hT}gz$y1W0^}rx2?K$Mq*KYIv{fUg zz#(NsjR@Fm^J?SWe+G{Vaw!{g(Ii+&A`*HNA%=vQq>H+L*}=c*F64r7{)4DJ^)0=O zvnrZ}z}>rTI*we%qX0wcg*;qLnqOGB_W7gpPi%fZ7@z~g3r+d%k!;L_tNBA>?GS4cnAQUxesC>HCNoGbS*}$HdDX=$pJioL{e@saN=J zZTHi5Fh)_V3UORX(Nm~|?kJFtbkV))L2|G`os6GpOx{R^b*~zS#u#91b**N{g@C9i zyr)S$(R6ycx+d^4Qkg$Ln9d%%lQ$b%!tg=mC)aRwS3x|ZmaC@=qWiocjnswAo6n*c<(NtbX0lA(CaP(_xa5v#5{{F#4IiRY1;6VT2wu}S!DU6aNb2OGN}@J>+}@UjY; zvp8%mES&e2`=mqg?PnTYC#tMk$|m>P!*H-?d%oA%#Mqv^qjVdk?|WpFbgH;$mX-NzdIKW9NM^Sc$$c z_7Y>IzELoX3?2OZ8Rza2G4$#0cuc!VVWw*qtswp41^;lM{0(Y)(DDG+;)C=whT800 zt_4aiJaf|!Gz^ufRKn0#71wU;Mth`u8+Mn6)6l;>2VOpz{S7@w|v`(ciM zcIKQQDOT50)_0)4KR^@8!Zr>-@EDxGeFzQ3Tvi|c46sciEt18_K+6BdWx{)Dl8x}! zcEp4P7C3l#L4u3KyyH;K83QZgE;v@&wsu^r^+QBXuCd#Nt~c#nFp|bC?un?RVB8OS ztjF{nN3Nj*{A4?C1)qXd4SWL?m7T3E)Yc9i!Fb#2r^l5#1Dbi?hkMFw(yV=ayjaeC z`dB8E=%c1@+i`>}#111@`&w?Yun7EHSJK#48W78pt9((_S{rH>m8#AG5kXXfaivAq zK#-Dzro{aLHdO{I=L+On6NZX=0j)GMZLMdf-5v6^fph7)8M(xt#DWZK)0S@wJf)Z4 zyQ~twck3-;$)N$aES5MY zq_J8vT6g#BoxKaPq^;1M5vp@t#@eEsZ*7M#9|wPY{=oHxj*xm#e!1?a$97|VEQ-Dp z6_Xt|+ZcD5A_L8*voxc+@EXB-3aTCuQUM(9 z>Us#8ujkVZPP>a;_sBRe)D5m-C?D0bd#GMc28(n2*(eZH8m#U3DdJ)p+y8ddOWO0+ zeTP3r5V$K}2Lzx^XH^CnuER$=>EfMFxQ`a`~6`QCmpW!L@DvQaN`!Ob%nYEeeB}3)Rvps1;yW0$2t@5QFXk02LSuyKPN> zcrus_+FKQqPrI^s^&zFsKTZB||Nf(=Zl};Cws`bwLsL_>CO5yr9v^Dp`t@T9zHIHw zeM77OeUwrA2?~Xx_ia$h^!%f@qs$&|D8+#*%Sgf0%G^t38&hExtx0P91)sowIK0ti zi@@jRo3iXE{IYi3GVrDbTUaPj*z8o(L~@IY8ttd*K+EfEPB}?+(g1<&q@9_GQ~3G^ z{om+JCW?}?e$sjCg@8vD7WWpje^+bb9dfC4PEAU^rEN4tLN)bb|4Fqp9O>K~eje-q z@AAfz?cEUP7fOcQC}RJER@&A3Y7<4jp&{4G5oy3~xrH-`;`g=$9*xQ(xVU(4O$$T) z1toj3mo*H?FCTq!3P^YtaNtzSMfn_c*7WAO1m~bapUWmM0qwNMRn$}o_uC@cW7p~I z+O=%p%7Fi@ybd2lYb%)_-!~(C{1SkDrM!X2CHbhn7W=Y7jNLsvs6toRN_Md}v5X zpFUlHj=C^<9mTA)!fkxEY5MfBYRG{{QVo~asL-`h zN;*YWqXDY6!PD5Af9$;owhG&7=%AF3cvg*2bk8(Eu{fzf4vdV%#l<~1I?5|_FNj{> znrRvXVY|&xmM{+a%O(|je#@#~ZVyp5%kQ3&iFB`AVV*xoNJtz55lrOqZ{M1FZ{7!Z zWlc9voG=of95+UY89E|lP5-v`HiNUe*%lJ?L^-0wn+i|51Ro;RR8 z;?%ErXgk}^Q%6JP(kTDtiDYs}y4s21;NtS_!1?)k;7YBHDOPJ~3O;xqW4VPf+CxxJ zc6h+#O5%pyR&XenxOsC_$#;VJ3d@w!&7Ca@__JUtQ)ST=4mkueK>d$ag?)jM0tFHB z`7StHPFZ1LSn1cLQu+PkRxUE#R26AWZlI%o{JWB}-Ms$>#=(CGRXck!JC}@2?5jrN z^}bJ?TK9>PUXm0(LGJQ9NEzHBBa?e%1|u!#;2;{fZBEKE?OzI6jg@1N-L-c(-w03e zk_qYV%$}BXTe^uvzvpDX8xR|k#n-%sEoF?U7i=N)Kd|4C(gbEAAA36uU4_DBRw+JA(u$4gZsO1Y3Y z1mD!Ur0DdZT7;p8v~&mDK58)!$EiAduuuS9&_%PX(xRbEYhUO9SRC#$e& z)A9QOflZo(b!vRPOhoC3^Nyh*A^{3g(xApYuw3w2?j;&6f?)~tPZ8Ae zjhITK!^6qlbk=U0 zdwD!_?fT6#kQNn$uCK2CTv-V(e`7>xa0(C>bo}Q_(GP&RtBm7Y;y9Ni*X_SHp@Mwkmb*@PTE`F9Qdbk zR7MGIuGKPv%&SYKa1 zRoq)wOKZRN)%BRZw|Dfio(S7d*3i8x^E26(da&Ls0d^q~_Cri$f~+>4gJL;>;SNA@ zDVi^k(`wPpltkqhT5i$h1eAX|xkBJJ=w@URjF)|QW}&9ZgI z%i!KQkEN)0CSJDSSOM)D0WR*dUD&`TvyuN<{CL`>ff`ViHO71pp8f=zh4$Rj!}XAH z$NoH(l$hu;laJWsU9{9M7sFWJ=fSr4^t3b@>nv6O@{-29p8DV6-lj^W60+IMLAJA? ziuM8hL$VhJ`ryU+sneCR`Wb*aboyN!1O9!nWUJ{5m4Xhy0QTvlEHn$1mz7xmwDTf4 zq*<$zR~#2Y>>KS>?lWyplWm-YAT-+VT9mx2VEzrZdr-w>W?oA=15XKE=)%m}0#--z zFNLiya>VoL#4|=qy&EUV@OWExi-Olr9(iljMnyz)V33^D4ij`NhPgGUTW?|H1gMw5 z(}LAufM6^p+)6<>PZVfmgE=N_7gC>+yPyX^zC$!~qb^6hgpUWS00u>W6JUEE=;ktubhD4Oh z<#Eg9LZv^yG$pHQ>Yt-22T+2ya#cdF;|PU2)|f9765X`) z4U3;q+}YNCam_wD;k6d*VTNuIdSsKv7kyn_K_?Bg4}7oW=kBAxR{6CpT1KJu0spR= z*akUmN`f=fFI`^^NrS(04@{jivw8XX?@t%>lgr6U{1ot0IuQ0`4LA3qfG#zxD5H>Ll(M_`%I*Ts3_i)Q`NjXS+z~S zHeJw`oq!uPFgVy1lG}CQkwvp|?IB0Qi&f2O4^7cN1b8w8<52}0%dL4pan(cX{S34z zqowgui>8)pBIJh~aNwnWfg=gb733o+%t1DmVsNx(8Tbv#V;lCb&o*VupBef1oO#2E zTL__(Q&=(Sd3=rX?^e^ch>(!Ff=#R7oX-<|+g1$Sl9*rtj3JV}LK<7&zXy65Ok-d&a4XLxRX8&52*JK$9lgj3PC;eHg^Zl^a`1gSgVl(FOKA!8C(y+wN%;?ybo{(iU z9cdr0bAU<}#jjKcXo1dVPO~t4@bkGogbG0pWTlV zVWou2jH)C389M@_pKeSUHC$`fZ;ultm;K(}kG5O)@bc>J?0o+tAUYu~PD@`u6x+*g zzt%bP%L`!(+hr{A97{+(p9XB7d&ii0BlJ2!il>)Suf9)iwY1id8gdVZHL1A3-ZxO zD#5MnXo(Zm_8ep!t>2kbZ+=9?=kpUOQu3g?eR&DV%()4CdEC4Q#mwo$U}M|e-3@Y4 z5$EMznAkeHNU;DXSwaAR%OP%ViISDBlIzb8hFwLQ2K$KBd$2ebW!NY*7`Wl(c`rdFCZ> z_Ti>_FGUGL{yuWFx+n7Xh-%z3RPQoGNEYQpf2Da-*e!s~?!5>VpfAW1Ap}1- zKN!jI+3TyH^A&{$BcMzi9du<>lVqL#7wl&8Bh)oCULmH-d9$8-1H&=!EggZ}uaJ*aB3kc9a zF{+*4cC>llv8~C+{_HxnZ-}O@;AsEp*RC3zHilX)V^**7)yIN=?JjBdR7pD6(4L{h z1L6K}v&R804J9Ltk%NPS3x;ULy=LF5G&8h#-3TCUWJPqgot@*^VKf!4ZI3H3g~!$_Z>r_LAHo z)j0lp_w1Z(6&K3n$n-xrGY=5n7SO4psM!c>z^>VWS_47QnLXb@@F1QK=}-9}2oSbQ zD2rLt(s6J9KH(`l)-W2QNYP+~z<9e<hc0i(RoTPVFcQ3SP)wxDMvao>s3V7DVdr$HDp2H`Zg=ZXtDXu#-osHE8r-i14B{3Qtj;4fs?hqwpIk4 zQ$fM|9B>a>0?Q&YPJLdNm4T~(*&=(WqEn2q7RMNJ_Og5>^YI#7lnPUAmDfKMD&??0 z`+l>wj<^^&SFc|6IzfW${K!#rYjy-tZ$BmQbd)Xi%_;Q<{WVoN zxx64YA_nJOa3wl|`k+JGNX+Timo%umx?6;Byr-w9ul$qR5CBE(dY%2*g_#D4xr|I_ zJi8XyMF?+5TKpDd7n5_0zeh4M`h;#QD=RBCm70?GHuS+@lamCXW6B2#^N^!I`a|-G%xK7Xi?)_Yj8(HD{1G|D9XzD17$qy=Yp3j};#1p^r z8Rv7aFc6`3?x#IG5Bbi?$*HdQ$@!a>>cIK?{|QS@b}9%>STcwVzs ze<+HMeh+0Y-gckl?Yk;K*G(_Ipm?$#NPKn1cHedl;`W^Pwzv%1P3DI$7A?HH>hU0b7%e4%MR@8>C&X%vF}@!brrfRuynN;V)J>z+`1RB7X-TGEwyDd zE32OX{4PCDh1eZ%zP6nr@AlO^PP)Y5r9D;oIt?-HZ6uBP&XA41ghYv-+h|0bR^U9w_t zG@4ybk2VR#5-Ezl5LW9H>w+-_KDIm15tkLQu^c+Zm4AUs;nPk#4rCR6W;IpSE8aRJ zp=NJkq>1 zXY^?|BF-amYGPsmf9(5rakq_eI6XnO9oAn`|K^vS^JX_}EybNb!;w$QFLT8G@Oudq zj$s6hHaQ+*T_y8N*J&;YsrSrJO)H1!`}R)O)`BOCf(^OD8?XPlGkRZOdM4JDuY9wO z_n1nck5*39ir7`9*V53?(87YTUx8$w|Jm^O@1)Eh0k(sf@aD}I0A@6rg4D8_uObZ) ziZn3N9{5SbJI_|Vb@^d`;yhLPPRG7`?dD)|4N+n(R_`3G45q;Bg*f6aaooFiA&@tnG?VZ%V^}6$xYOyG zJ4$TXLt0m}H5}BBxpKTo_U{2#%OCGWJa*<`FednH+`r^5EW=@D&V?)(l7i?PV_k^p zfGc4|Wxcc!J}m!D;RL<9pg=*4^YW?hwASX1}8Kj_jiaxXX%nK1a{eSl?7a zmJbv^EiliG$C2XsaZqIZYh%8({ZCE*bWrAGw1^7Lc3nk%>82!Uj`vOJL=Z5ZsKl9H zFhJIfuCqe<^5*4AsypRaq~A$7ub<3@YOIPc>wS;&n_NwVpPimLjcHgjrOZ_2+Q%522%XP6ok*&! z-)MgMZDZd}$W@gZZLJ32<41BP~BtrwH zq~|z$lYkygxAlR7#j>QO=It{$4FvXAS^lOe7kUzj zAq()(oWO9%YZ1!5m>oI;bD^8WaBZc?DgnZ2cko z{!_&i1lMCB!>Mt{a-PJ$`Ouz(=0B#{T85QRPQq*31+Gu^yN<7v%vLxr zC48q6T?X zdM*b}`wjliF}|H7YQ)^dvDybW$T}Zv3Y2$QJ)nfSsiwNglK=V14WB+&aiND(2kfghFU z6hmwpoK0pz#I~cuqWYgQ7lvD#FA2KfOFvKLew1cvbCa;1PhkGWn;7r} zpEbj{?%6~DIu2Y-^o+As#x?HToK3p=hm3XC>O38Ld7fDuB{iiB-gR|zJKEiq+>zv1q_N^zYuatox#O7z;S*yGd*QSE zhB8!VRk#u76y64r-}SQy9^NAU8Fo<{rJ5m`?fVX1rC8jGk!;Az-F}H@A14^Fri&XY zS(Otj92eT_q@faY`gQvEulV7pmiN)ds*EBu9stlsMYr)IC_8^?A$;;k?cV1qey5eZq;lY9W5lq_AR52@GQ@3*g zABSKD&y7~t`NlPMCI5hST8+x?RYc4vW>`Cw4))WK97byV+jMl%Pcamoo`&bpmMTC-p@$#*(a?g>eXfd1h$u$j zPgbN;d07QWG}o`DLNvZSjX( z(r^aE&BMnC+&*bW+ZhE)wV^sWx0-2&V(RTrW({$}4?`aOkWUXXr&V%pH46QitZ>dX z^sA(rdt{@W-sS~KssnM*aD(YV zcD12$V2t)d^4xJS4UGt)^6i9QV#-v&##Q z;xTc(a}TQw#Q+_(XTCCr-QT|LK*uRHC~ss0vs>GR_8@3%Egp5g{JpvP(du&oyH?&n z(mf#GUw5|+9Qdl;zzQe|WNG=93&2cozp_mZgB+2`C)ehvr?gHU^p68?T9%c!-3b!0#m~V zYQ_v<7ZwT1Bl?@K=^prs4G?DFmuo;JQ6dv`vT*iyA1d3P2bX7nNyT;h(^J(^d_po# z6}*F4_3p0>zi#=5GpJt}Yjw~&c}yk%=Ny??T|6HLgeH%?d0S^#XwUW&#;EV(+1zKQ zr+Ya7JZ4DnFk6Vl_{iTvwE@PX?HZz@qT@_OI6_2+H)R4)qV)9t1PNM__A742fVYJQ z8soLmw6}^*PFnvS4IJ6DCY^9HE6v|%Op*D7CoS)N!hr0t5P<6H>5<<-j=t2G0Fz{K zr^U|7$35@kPmi~OQ!@)8yTo}UPo-coD?wTMAmyEcxU+Lv<|*g(u_Wt9)@iOO_P>3- zMStBL``zBWPhP*8vMu!3>Guv1P_k%f(xBZ~Bxb8~6gDQ&*@wXe<83J|1Q&l$PS!uf z-CF8HBej(5IQR85Bdolv_V+4ANfXG(UzzwW)^jyAVp0~O%<$@8t4s1ktaCh_H&9VI z9OQC!Mco|m^3WsTA}Q>68fW=MC(*_Bk(G=sr9`2Uo1?{Fm272GAJ)+$p-!hY^?X~U z4z)bJPW=Op3hIHt4U}KEg*BSHyi1 z@8BG0<-_pJuz-btb`^+w9B5JQT7DQDD_v1>qj*@@VAMfN2y_SV#04*ioOn39Z0Y2?h_Z%==}HQal#N64Gdr!3P+FM870yZqb4FM3Xv4p>#1O3 z_XC_@LjWs%14%XnN+^#V`n3`7<1L+p3i#(E_g1@&HijDyd2e*HSI7gmh>&z}{R!YP zi;Dr8SnoPySM!w(ODL>jc%_8~%cajxa(vr-Vt(R(_*p)N520dC=-mVrBi?)zB~q3D z4(BR!-BRBh+1j74f_6uPZBtiq4i(G$OW*Mu>i!tRWa90+hZHT$R5R&fjdI8#*FlvP z1qgP(vogO-dZG5zb;Ir$M`AUC{M~Jc-D&7XUFF&$t) zxZnUYxtt8%`6s*sVC_j%%RU_ftt?!YKxGbg_WS*uazWu_8#wY;^8$eS=UL?K6`!Fz zP6q%%*zLRp&at7?6L=v2s8Kam5}{;v!D~nAkRWLh#vyW7(7GLpKf}<#VBt52%vVEv z3%>?%R)e>7FN3pjT+Z&u&`@mhZ6eyD$X<>04!gBdAe^fD=!auOqua2ey(gPA^zd=> zA^j?G%2~xVD}`d{zzi(^eQ}wYh~L3dNUzXm84(p{k&$zRT-(NOlAe|O5u~}88JJIe zKp^V?I5(}Dqm<=RYisKrV(z&?9W0@6Gj;V;sUr#+ny8*zj50O+`S`Rt9*&rsF`D=j za{tDenWWVAHDL~hJ=L!3Qg!i?;yPz>xgINyVt;K79#f{ONn8O#jaA^N`ZS|LV$=yl zT+jltMt%*sJxuu0^>)Rr5W@M zt_9g8RtbLQ_MVO!SuSpvSgc=2DFt>NJYUwnPaE$&eu^kf)A5U*YF3gDQinwcXYqH?{_fx-9dw6jQfoyT-NNrqS-+lY$_MSr8T2CLqbOyM z(A7`9)=TTteNSvh-#AK_8B{;{v$Fs$=sDR9*8#zUxyUt!_~fWyxr?Xd5y$2k|w9ddy*rVy`_ad?fIBRhL z4$I(LXTT(!+n z_pj5Y;|S8LRPbj)|IbhTjW~34bSRu1#U$A^7O{1)s3H>x*{`W(th)z_YY-E%2ndi7 zNTez}I7dAkHd8Q`VrxMUc~&Y~JpI`@*gWw_%S6XvJHvI`F+89lK9*8|eQOoeg$hip zK(V}$L2^^;#*G_Ssz>$(Ux1_mFBK3F(ACuia39Ln0((GMBzf>;=egV#3DM{&AOtxv zKK;ngLPuos$tP*WHJdrF6Ce4~ygr9SxH&uAIZg=^4>_W78pD~Kh$K=pBj0AgD zVK~twsaT=(jlDF>-#<&yOBea#4Gb-p7pJhThS)V9z(hc$qNxPGjE(6Dp}XdTYw!WM z*tec@z6n=$o^O2xU;(k4ViJPGFdF&rQwLu34kV!yfFmuX1(5s%*G`VNAHZ}j&n*$r z&Xg^w!VpLI@?Z$YqOsMljhXC%du=dbS!}aqAIf$lPyBSF8q@zy9hvWJv*-AAwTmr$ zz>q0X^}vx(eR(`^9;Bi7>Bnq8eU5MS6VQpKR-Jo4CBp_qKKQzk`7N{^iGhgyJktO zn3v}4O_(=Q`hJC_ZE+O%5@sm?%;URj5>pCXx)>Ogq{aCNn(`oV?x=dsSzq6@)sPRe zHz{C=)DZr%L3GM4HBZjNeQ#hV_!c+**z+H`*L}Fc|oUAITDOyG~Bp4 z3i)$*Rer3WJXd@ny88EmFwY&MjkPfUnhz?OO=B%i6Qu)yf|Aq^n)jhejL)a9g{W)v zCiqwHbmsj0pg>#2U0AyA8qGPODd$E>ySBkmqn5=_e2>*j+c#ziRo_yo%czvf11$ie=y~;In;RwaB!p%S3MqbiD7k2nLQ*v%{afF zRJ_zSzan3jhE(9K$NE9P@shHsewL6}BvEV}Kcj#11FQOM`#71mwlLU-$synU`=qLp zs-72re=R)X&rzRGX9*kY#l+m{W6d}_+yDB9JRHac98{>InkOjWciC?)sjIey0q+&& zw}bxIu0xIWFr*hZ7Gsqw{U1U@k7odZZf3H`v&WIN2LcuMaWpR$*T|rGoVNbtc3gY- zLn~rMM2rsh;9D7ArEd%zr4ify6a~vHuX5)6Cm3!AYl!u8tSgJ(`N3@CfF-ega_WKm63 zIDEJ~2nC<%GnD2+We-rpA>xAvI65ECPlMEmgZ=5!nOg!#^nBSQl78<>P0d1{D@Be} zM6SO+IXYqR${$8olBw*HhgH&ixY@i1=^u=YWOP}qM4PbFOkbz-fJ&2dkzaz5nHgO* z5&qCKz#ZMf-zBcNB_E_gTG4ZG@B(Tn1$rW`!%(70hAH86so6Qk?D-mdlnSbg^vZS` z5G}vGK2{*|=DfXkHBI8-;VCSb9BCr;TmWxNh&x~Dv3Nhi@CHg6hWzOf1kpcXw?$sC zsisQ2JOLytD82Z`9^bPaD4jJmG$sJ#y5hEsKaK{z!SE1JMOv?4neIdX1yfwcQwWXt z>`6dh0)%MR_tYLCclB^m5cmE8i)wF$uOW244M6D?g2_ZRR6uu>hdW-}fpnGegOWAZ zM>FxQ{Ij=ji8*BA=b|cF_VBfzXjQ5(yL(fka3B|Xv3;BdQw`hHc>9c;-8hR}7#+#r zQJu&KvfIQ<;AQv5#Bc{3VM^~3QRL6dRLN(W;+~fvbBUw-tQ)Z!WHT)tyFdv$87S(v zHd+V_o~N3cnsJ>33?7b3`V$rA9dgmM=Ri)tZe+g60Q6&qT>kGIucm}pSZ-PmCf&oR zcL(R^QKEOE|EhMt+;f`Xxd6Z+PQXxGSg41vPSjY_9vluejz7o{?rLB!8e=0gPn|Y@gA&;_>DXN#kV15im|x&^(WdU1FLhC~ z8KPM4r?_Wkx=60pB0_nZ7T(b3^n6EL#kvwN3$5LA&KTG-#k z>$TwLG==$5MrsYUv(k6=p+n2qP<9yUtQ#E~16+u4MjyrK7Q z+|X$_dqNScVK%uGC-wCm8HQGTeVB&=dYbpwT@vG)cRylG?|>ltYG(`#M$DrjTTt`c ze1o&_3Dyq*R_&SpMuUGGkwg|>g;)8Epj4J^o1mW;cjRz>NcYhQJq)!)4ezMuczwcr zX=F&PtihZW)kb0;nLlq4x|#+YC4ED~Ettl`@C?R8v$M0k_%O-O_6G@;-$?Sc3+d(a zN{V3fEvBH{+Wz3mJylsj**Y1U(yvOQ7+N)V)3c)J_=-!|ixME6{f|(ybQrNHj6v>Ts-FYh~$!FYuTXJnh}PV}k6Wf&wAfs{thxsN#m1Z68y}y@3YAr^no2E#lS@8KtjC|4IZWd1H(qZOLJ{7T?EAIDyc<9 zMm8esaXWg6{{3ZE#o+0UWtEhiP1==DOie)*T9$>s)C=0JK|}Esy@O1+JEGt zx{(o12uZLaNC3^$UcMAJfu&&W1iOaRpDHUggkHx~v4j;6zR$`+GbemVu^yU+$z&Y5 zXzRqAQ-Wl~&1zo_)fGVg<4vfOdqd`qvFr~7J1ymS*cc|~8CoQ5suw zS0xT4{2r}PtQYH+CIi~VxO;R&R{x%{9hSjE#QFkTzTj>FUI9(-3x$4!c_09iFxm$c z`n3z=7q}{5yrFL9;{#Gb0NH$enxn$P+A!LVW8S_EA_psRD74@^E?94o+{N8j1JOU6 zbc*0r?h@0Miy?VBFbW7tX`7Zf+v$MzdKW4WpFayBP=4)ylO~WPfk{9i5y6jR0A>g% zc>VFOFk28lZH|>^n4w^1bX5o+MN;0!mM2<)l`lvIg;oZe@lYZrvwT?G<$;qXO<{Zv zlS@q`iVvy32xN+w^b2CbUl3%0pdq)v`w&wtRp_qonFj)ixdUT`^=BFtYHe zsh{=2!N(E}y4eCeESQ13n1LMnPzK5$6&SuY2f2JYKbnGP-~-2-NLYfz0=7MDfGmV| zlrF?PvFU$*+ume^50-$9mDSt0xN4uj&Nfp()CUnG;U5@uIW|KY;^5`wWoNgAT}VRV zJC-QNd(@QTUNvE1Nbu%?rOq zs6;My0`q`?lHUeuJgC%mv=diM~8;m+z)|Urk>5*!An5#KbPsJ&Y`Y^ z;D3&5%>6!qH6}lQj_AgQhVJ&-A=KsFU0q z`}!A=?j#S+lQ;iNyL@Fht0$fX06;SEX4@iAF;Wn5Ti>#nu1LNBQ3nl0YJwXEc&v07 zD1;HVe)~q~jJX~W8meFWlyvkM=zr}R`s>Y}`zBB_0T(F_DSBWuuKR}piRSNjMUc5A%KSR;P3J+50W$R(t*hX+zdnabKSnSspw;NYM;n(h)hPgpS)`s?}O-d;P#?fXwhVL^;-W-oB3ptW-Rnl4^b#hg?T zkHiNU2-I&1ViK4nC%Hfe2M=y&zPEqPy*5uw0Pm6i237FzGmrcV#Bhjj3(mYBA)2Cs z=~2k`MH{qqu#^LH`fGXhfG##LBPwcYNG`&50M_u<)FhTU08*MCgkB)0<4?pXqQ((K ztj12jI#&}vBzZwJ#26&xG+AS7nVFM39(ewT6LUZ3HPElx&O2Gc>SOeCF5#HeS5&lq z09Sy^ld+a`k3}rDEgj9`!s{zm*Rjzxt}P_U;b(*@)~$AI3nyQUyEvjAYD5qLHZ zZ4nHhu(7d^w&xzeK-Z9s*t837wORDeTAnCNm>Nci57a3(G)jN$Vg7vmPZKXL6V89j zv-I}i$(M3mFEA|=zg#QNE=xJ6gWXbfb-Er?i|G&(;ebGAvGLl7(Vv&0h>&#@LOBs5 z>zmtdtfVfkuC*S~jw0v@lUPB4MvL)e4={%VBc$+ikC=8Vj2VJFUezxu?CfTK1NJ!y z#aM1@yHFprJU%h7KH50ztwXGNBF44 Date: Fri, 14 Nov 2025 19:05:11 +0800 Subject: [PATCH 09/31] [FLINK-33392][docs] Add the documentation pages for balanced tasks scheduling. (#27241) (cherry picked from commit 7de3392b47096fb0f4163b9217a5d344b7361791) --- .../deployment/tasks-scheduling/_index.md | 23 + .../balanced_tasks_scheduling.md | 123 ++++ .../deployment/tasks-scheduling/_index.md | 23 + .../balanced_tasks_scheduling.md | 123 ++++ ...s_to_taskmanagers_allocation_principle.svg | 451 +++++++++++++++ .../tasks_scheduling_skew_case.svg | 246 ++++++++ .../tasks_to_slots_allocation_principle.svg | 531 ++++++++++++++++++ 7 files changed, 1520 insertions(+) create mode 100644 docs/content.zh/docs/deployment/tasks-scheduling/_index.md create mode 100644 docs/content.zh/docs/deployment/tasks-scheduling/balanced_tasks_scheduling.md create mode 100644 docs/content/docs/deployment/tasks-scheduling/_index.md create mode 100644 docs/content/docs/deployment/tasks-scheduling/balanced_tasks_scheduling.md create mode 100644 docs/static/fig/deployments/tasks-scheduling/slots_to_taskmanagers_allocation_principle.svg create mode 100644 docs/static/fig/deployments/tasks-scheduling/tasks_scheduling_skew_case.svg create mode 100644 docs/static/fig/deployments/tasks-scheduling/tasks_to_slots_allocation_principle.svg diff --git a/docs/content.zh/docs/deployment/tasks-scheduling/_index.md b/docs/content.zh/docs/deployment/tasks-scheduling/_index.md new file mode 100644 index 0000000000000..06fbdd801c4a3 --- /dev/null +++ b/docs/content.zh/docs/deployment/tasks-scheduling/_index.md @@ -0,0 +1,23 @@ +--- +title: Tasks Scheduling +bookCollapseSection: true +weight: 9 +--- + diff --git a/docs/content.zh/docs/deployment/tasks-scheduling/balanced_tasks_scheduling.md b/docs/content.zh/docs/deployment/tasks-scheduling/balanced_tasks_scheduling.md new file mode 100644 index 0000000000000..7006cc56e0c88 --- /dev/null +++ b/docs/content.zh/docs/deployment/tasks-scheduling/balanced_tasks_scheduling.md @@ -0,0 +1,123 @@ +--- +title: Balanced Tasks Scheduling +weight: 5 +type: docs + +--- + + +# Balanced Tasks Scheduling + +This page describes the background and principle of balanced tasks scheduling, +how to use it when running streaming jobs. + +## Background + +When the parallelism of all vertices within a Flink streaming job is inconsistent, +the [default strategy]({{< ref "docs/deployment/config" >}}#taskmanager-load-balance-mode) +of Flink to deploy tasks sometimes leads some `TaskManagers` have more tasks while others have fewer tasks, +resulting in excessive resource utilization at some `TaskManagers` +that contain more tasks and becoming a bottleneck for the entire job processing. + +{{< img src="/fig/deployments/tasks-scheduling/tasks_scheduling_skew_case.svg" alt="The Skew Case of Tasks Scheduling" class="offset" width="50%" >}} + +As shown in figure (a), given a Flink job comprising two vertices, `JobVertex-A (JV-A)` and `JobVertex-B (JV-B)`, +with parallelism degrees of `6` and `3` respectively, +and both vertices sharing the same slot sharing group. +Under the default tasks scheduling strategy, as illustrated in figure (b), +the distribution of tasks across `TaskManagers` may result in significant disparities in task load. +Specifically, the `TaskManager`s with the highest number of tasks may host `4` tasks, +while the one with the lowest load may have only `2` tasks. +Consequently, the `TaskManager`s bearing 4 tasks is prone to become a performance bottleneck for the entire job. + +Therefore, Flink provides the task-quantity-based balanced tasks scheduling capability. +Within the job's resource view, it aims to ensure that the number of tasks +scheduled to each `TaskManager` as close as possible to, +thereby improving the resource usage skew among `TaskManagers`. + +Note The presence of inconsistent parallelism does not imply that this strategy must be used, as this is not always the case in practice. + +## Principle + +The task-quantity-based load balancing tasks scheduling strategy completes the assignment of tasks to `TaskManagers` in two phases: +- The tasks-to-slots assignment phase +- The slots-to-TaskManagers assignment phase + +This section will use two examples to illustrate the simplified process and principle of +how the task-quantity-based tasks scheduling strategy handles the assignments in these two phases. + +### The tasks-to-slots assignment phase + +Taking the job shown in figure (c) as an example, it contains five job vertices with parallelism degrees of `1`, `4`, `4`, `2`, and `3`, respectively. +All five job vertices belong to the default slot sharing group. + +{{< img src="/fig/deployments/tasks-scheduling/tasks_to_slots_allocation_principle.svg" alt="The Tasks To Slots Allocation Principle Demo" class="offset" width="65%" >}} + +During the tasks-to-slots assignment phase, this tasks scheduling strategy: +- First directly assigns the tasks of the vertices with the highest parallelism to the `i-th` slot. + + That is, task `JV-Bi` is assigned directly to `sloti`, and task `JV-Ci` is assigned directly to `sloti`. + +- Next, for tasks belonging to job vertices with sub-maximal parallelism, they are assigned in a round-robin fashion across the slots within the current +slot sharing group until all tasks are allocated. + +As shown in figure (e), under the task-quantity-based assignment strategy, the range (max-min difference) of the number of tasks per slot is `1`, +which is better than the range of `3` under the default strategy shown in figure (d). + +Thus, this ensures a more balanced distribution of the number of tasks across slots. + +### The slots-to-TaskManagers assignment phase + +As shown in figure (f), given a Flink job comprising two vertices, `JV-A` and `JV-B`, with parallelism of `6` and `3` respectively, +and both vertices sharing the same slot sharing group. + +{{< img src="/fig/deployments/tasks-scheduling/slots_to_taskmanagers_allocation_principle.svg" alt="The Slots to TaskManagers Allocation Principle Demo" class="offset" width="75%" >}} + +The assignment result after the first phase is shown in figure (g), +where `Slot0`, `Slot1`, and `Slot2` each contain `2` tasks, while the remaining slots contain `1` task each. + +Subsequently: +- The strategy submits all slot requests and waits until all slot resources required for the current job are ready. + +Once the slot resources are ready: +- The strategy then sorts all slot requests in descending order based on the number of tasks contained in each request. +Afterward, it sequentially assigns each slot request to the `TaskManager` with the smallest current tasks loading. +This process continues until all slot requests have been allocated. + +The final assignment result is shown in figure (i), where each `TaskManager` ends up with exactly `3` tasks, +resulting in a task count difference of `0` between `TaskManagers`. In contrast, the scheduling result under the default strategy, +shown in figure (h), has a task count difference of `2` between `TaskManagers`. + +Therefore, if you are seeing performance bottlenecks of the sort described above, +then using this load balancing tasks scheduling strategy can improve performance. +Be aware that you should not use this strategy, if you are not seeing these bottlenecks, +as you may experience performance degradation. + +## Usage + +You can enable balanced tasks scheduling through the following configuration item: + +- `taskmanager.load-balance.mode`: `tasks` + +## More details + +See the FLIP-370 for more details. + +{{< top >}} diff --git a/docs/content/docs/deployment/tasks-scheduling/_index.md b/docs/content/docs/deployment/tasks-scheduling/_index.md new file mode 100644 index 0000000000000..06fbdd801c4a3 --- /dev/null +++ b/docs/content/docs/deployment/tasks-scheduling/_index.md @@ -0,0 +1,23 @@ +--- +title: Tasks Scheduling +bookCollapseSection: true +weight: 9 +--- + diff --git a/docs/content/docs/deployment/tasks-scheduling/balanced_tasks_scheduling.md b/docs/content/docs/deployment/tasks-scheduling/balanced_tasks_scheduling.md new file mode 100644 index 0000000000000..7006cc56e0c88 --- /dev/null +++ b/docs/content/docs/deployment/tasks-scheduling/balanced_tasks_scheduling.md @@ -0,0 +1,123 @@ +--- +title: Balanced Tasks Scheduling +weight: 5 +type: docs + +--- + + +# Balanced Tasks Scheduling + +This page describes the background and principle of balanced tasks scheduling, +how to use it when running streaming jobs. + +## Background + +When the parallelism of all vertices within a Flink streaming job is inconsistent, +the [default strategy]({{< ref "docs/deployment/config" >}}#taskmanager-load-balance-mode) +of Flink to deploy tasks sometimes leads some `TaskManagers` have more tasks while others have fewer tasks, +resulting in excessive resource utilization at some `TaskManagers` +that contain more tasks and becoming a bottleneck for the entire job processing. + +{{< img src="/fig/deployments/tasks-scheduling/tasks_scheduling_skew_case.svg" alt="The Skew Case of Tasks Scheduling" class="offset" width="50%" >}} + +As shown in figure (a), given a Flink job comprising two vertices, `JobVertex-A (JV-A)` and `JobVertex-B (JV-B)`, +with parallelism degrees of `6` and `3` respectively, +and both vertices sharing the same slot sharing group. +Under the default tasks scheduling strategy, as illustrated in figure (b), +the distribution of tasks across `TaskManagers` may result in significant disparities in task load. +Specifically, the `TaskManager`s with the highest number of tasks may host `4` tasks, +while the one with the lowest load may have only `2` tasks. +Consequently, the `TaskManager`s bearing 4 tasks is prone to become a performance bottleneck for the entire job. + +Therefore, Flink provides the task-quantity-based balanced tasks scheduling capability. +Within the job's resource view, it aims to ensure that the number of tasks +scheduled to each `TaskManager` as close as possible to, +thereby improving the resource usage skew among `TaskManagers`. + +Note The presence of inconsistent parallelism does not imply that this strategy must be used, as this is not always the case in practice. + +## Principle + +The task-quantity-based load balancing tasks scheduling strategy completes the assignment of tasks to `TaskManagers` in two phases: +- The tasks-to-slots assignment phase +- The slots-to-TaskManagers assignment phase + +This section will use two examples to illustrate the simplified process and principle of +how the task-quantity-based tasks scheduling strategy handles the assignments in these two phases. + +### The tasks-to-slots assignment phase + +Taking the job shown in figure (c) as an example, it contains five job vertices with parallelism degrees of `1`, `4`, `4`, `2`, and `3`, respectively. +All five job vertices belong to the default slot sharing group. + +{{< img src="/fig/deployments/tasks-scheduling/tasks_to_slots_allocation_principle.svg" alt="The Tasks To Slots Allocation Principle Demo" class="offset" width="65%" >}} + +During the tasks-to-slots assignment phase, this tasks scheduling strategy: +- First directly assigns the tasks of the vertices with the highest parallelism to the `i-th` slot. + + That is, task `JV-Bi` is assigned directly to `sloti`, and task `JV-Ci` is assigned directly to `sloti`. + +- Next, for tasks belonging to job vertices with sub-maximal parallelism, they are assigned in a round-robin fashion across the slots within the current +slot sharing group until all tasks are allocated. + +As shown in figure (e), under the task-quantity-based assignment strategy, the range (max-min difference) of the number of tasks per slot is `1`, +which is better than the range of `3` under the default strategy shown in figure (d). + +Thus, this ensures a more balanced distribution of the number of tasks across slots. + +### The slots-to-TaskManagers assignment phase + +As shown in figure (f), given a Flink job comprising two vertices, `JV-A` and `JV-B`, with parallelism of `6` and `3` respectively, +and both vertices sharing the same slot sharing group. + +{{< img src="/fig/deployments/tasks-scheduling/slots_to_taskmanagers_allocation_principle.svg" alt="The Slots to TaskManagers Allocation Principle Demo" class="offset" width="75%" >}} + +The assignment result after the first phase is shown in figure (g), +where `Slot0`, `Slot1`, and `Slot2` each contain `2` tasks, while the remaining slots contain `1` task each. + +Subsequently: +- The strategy submits all slot requests and waits until all slot resources required for the current job are ready. + +Once the slot resources are ready: +- The strategy then sorts all slot requests in descending order based on the number of tasks contained in each request. +Afterward, it sequentially assigns each slot request to the `TaskManager` with the smallest current tasks loading. +This process continues until all slot requests have been allocated. + +The final assignment result is shown in figure (i), where each `TaskManager` ends up with exactly `3` tasks, +resulting in a task count difference of `0` between `TaskManagers`. In contrast, the scheduling result under the default strategy, +shown in figure (h), has a task count difference of `2` between `TaskManagers`. + +Therefore, if you are seeing performance bottlenecks of the sort described above, +then using this load balancing tasks scheduling strategy can improve performance. +Be aware that you should not use this strategy, if you are not seeing these bottlenecks, +as you may experience performance degradation. + +## Usage + +You can enable balanced tasks scheduling through the following configuration item: + +- `taskmanager.load-balance.mode`: `tasks` + +## More details + +See the FLIP-370 for more details. + +{{< top >}} diff --git a/docs/static/fig/deployments/tasks-scheduling/slots_to_taskmanagers_allocation_principle.svg b/docs/static/fig/deployments/tasks-scheduling/slots_to_taskmanagers_allocation_principle.svg new file mode 100644 index 0000000000000..9a9e4ccab21ac --- /dev/null +++ b/docs/static/fig/deployments/tasks-scheduling/slots_to_taskmanagers_allocation_principle.svg @@ -0,0 +1,451 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/docs/static/fig/deployments/tasks-scheduling/tasks_scheduling_skew_case.svg b/docs/static/fig/deployments/tasks-scheduling/tasks_scheduling_skew_case.svg new file mode 100644 index 0000000000000..123c9b43944f4 --- /dev/null +++ b/docs/static/fig/deployments/tasks-scheduling/tasks_scheduling_skew_case.svg @@ -0,0 +1,246 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/docs/static/fig/deployments/tasks-scheduling/tasks_to_slots_allocation_principle.svg b/docs/static/fig/deployments/tasks-scheduling/tasks_to_slots_allocation_principle.svg new file mode 100644 index 0000000000000..c35f4f429d9be --- /dev/null +++ b/docs/static/fig/deployments/tasks-scheduling/tasks_to_slots_allocation_principle.svg @@ -0,0 +1,531 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + From ca9f7b7595943dc722aaf3a407d9a58cbbf49c0d Mon Sep 17 00:00:00 2001 From: Sergey Nuyanzin Date: Fri, 14 Nov 2025 21:46:50 +0100 Subject: [PATCH 10/31] [hotfix][docs] Add `SHOW MATERIALIZED TABLES`, `SHOW CREATE MATERIALIZED TABLE` to doc --- docs/content.zh/docs/dev/table/sql/show.md | 25 +++++++++++++++++++ docs/content/docs/dev/table/sql/show.md | 29 ++++++++++++++++++++-- 2 files changed, 52 insertions(+), 2 deletions(-) diff --git a/docs/content.zh/docs/dev/table/sql/show.md b/docs/content.zh/docs/dev/table/sql/show.md index 5b478a89d5539..d8f631eb451ba 100644 --- a/docs/content.zh/docs/dev/table/sql/show.md +++ b/docs/content.zh/docs/dev/table/sql/show.md @@ -39,6 +39,8 @@ SHOW CREATE 语句用于打印给定对象的创建 DDL 语句。当前的 SHOW - SHOW CURRENT DATABASE - SHOW TABLES - SHOW CREATE TABLE +- SHOW MATERIALIZED TABLES +- SHOW CREATE MATERIALIZED TABLE - SHOW COLUMNS - SHOW PARTITIONS - SHOW PROCEDURES @@ -975,6 +977,29 @@ The syntax of sql pattern in `LIKE` clause is the same as that of `MySQL` dialec SHOW CREATE VIEW [catalog_name.][db_name.]view_name ``` +## SHOW MATERIALIZED TABLES + +```sql +SHOW MATERIALIZED TABLES [ ( FROM | IN ) [catalog_name.]database_name ] [ [NOT] LIKE ] +``` + +Show all materialized tables for an optionally specified database. If no database is specified then the materialized tables are returned from the current database. Additionally, the output of this statement may be filtered by an optional matching pattern. + +**LIKE** +Show all materialized tables with given materialized table name and optional `LIKE` clause, whose name is similar to the ``. + +The syntax of sql pattern in `LIKE` clause is the same as that of `MySQL` dialect. +* `%` matches any number of characters, including zero characters, `\%` matches one `%` character. +* `_` matches exactly one character, `\_` matches one `_` character. + +## SHOW CREATE MATERIALIZED TABLE + +```sql +SHOW CREATE MATERIALIZED TABLE [catalog_name.][db_name.]materialized_table_name +``` + +Show create materialized table statement for specified materialized table. + 展示创建指定视图的 create 语句。 ## SHOW FUNCTIONS diff --git a/docs/content/docs/dev/table/sql/show.md b/docs/content/docs/dev/table/sql/show.md index 5231dd73680e3..bd6454ff069bc 100644 --- a/docs/content/docs/dev/table/sql/show.md +++ b/docs/content/docs/dev/table/sql/show.md @@ -43,6 +43,8 @@ Flink SQL supports the following SHOW statements for now: - SHOW PROCEDURES - SHOW VIEWS - SHOW CREATE VIEW +- SHOW MATERIALIZED TABLES +- SHOW CREATE MATERIALIZED TABLE - SHOW FUNCTIONS - SHOW MODULES - SHOW JARS @@ -642,8 +644,8 @@ show create catalog cat2; SHOW DATABASES [ ( FROM | IN ) catalog_name] [ [NOT] (LIKE | ILIKE) ] ``` -Show all databases within optionally specified catalog. -If no catalog is specified, then the default catalog is used. +Show all databases within optionally specified catalog. +If no catalog is specified, then the default catalog is used. Additionally, a `` can be used to filter the databases. **LIKE** @@ -986,6 +988,29 @@ SHOW CREATE VIEW [catalog_name.][db_name.]view_name Show create view statement for specified view. +## SHOW MATERIALIZED TABLES + +```sql +SHOW MATERIALIZED TABLES [ ( FROM | IN ) [catalog_name.]database_name ] [ [NOT] LIKE ] +``` + +Show all materialized tables for an optionally specified database. If no database is specified then the materialized tables are returned from the current database. Additionally, the output of this statement may be filtered by an optional matching pattern. + +**LIKE** +Show all materialized tables with given materialized table name and optional `LIKE` clause, whose name is similar to the ``. + +The syntax of sql pattern in `LIKE` clause is the same as that of `MySQL` dialect. +* `%` matches any number of characters, including zero characters, `\%` matches one `%` character. +* `_` matches exactly one character, `\_` matches one `_` character. + +## SHOW CREATE MATERIALIZED TABLE + +```sql +SHOW CREATE MATERIALIZED TABLE [catalog_name.][db_name.]materialized_table_name +``` + +Show create materialized table statement for specified materialized table. + ## SHOW FUNCTIONS ```sql From f621ab665cc8c93bd95084750361a235aad2c4ea Mon Sep 17 00:00:00 2001 From: Ferenc Csaky Date: Mon, 17 Nov 2025 12:41:20 +0100 Subject: [PATCH 11/31] [FLINK-38682][table-planner] Support unknown -> RAW cast during type inference in limited validator scope --- .../TypeInferenceOperandChecker.java | 32 ++++++++-- .../runtime/batch/sql/FunctionITCase.java | 64 ++++++++++++++++++- 2 files changed, 91 insertions(+), 5 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandChecker.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandChecker.java index 467e2178cccc4..e20c2c0754835 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandChecker.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandChecker.java @@ -19,10 +19,12 @@ package org.apache.flink.table.planner.functions.inference; import org.apache.flink.annotation.Internal; +import org.apache.flink.sql.parser.type.SqlRawTypeNameSpec; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.DataTypeFactory; import org.apache.flink.table.functions.FunctionDefinition; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; +import org.apache.flink.table.planner.plan.schema.RawRelDataType; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.inference.ArgumentCount; import org.apache.flink.table.types.inference.CallContext; @@ -32,16 +34,20 @@ import org.apache.flink.table.types.inference.TypeInference; import org.apache.flink.table.types.inference.TypeInferenceUtil; import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RawType; import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rel.type.StructKind; import org.apache.calcite.sql.SqlCallBinding; +import org.apache.calcite.sql.SqlDataTypeSpec; import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlLiteral; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlOperandCountRange; import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.SqlTypeNameSpec; import org.apache.calcite.sql.fun.SqlStdOperatorTable; import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.calcite.sql.type.SqlOperandMetadata; @@ -240,10 +246,28 @@ private void insertImplicitCasts(SqlCallBinding callBinding, List expe /** Adopted from {@link org.apache.calcite.sql.validate.implicit.AbstractTypeCoercion}. */ private SqlNode castTo(SqlNode node, RelDataType type) { - return SqlStdOperatorTable.CAST.createCall( - SqlParserPos.ZERO, - node, - SqlTypeUtil.convertTypeToSpec(type).withNullable(type.isNullable())); + final SqlDataTypeSpec dataType; + if (type instanceof RawRelDataType) { + dataType = createRawDataTypeSpec((RawRelDataType) type); + } else { + dataType = SqlTypeUtil.convertTypeToSpec(type).withNullable(type.isNullable()); + } + + return SqlStdOperatorTable.CAST.createCall(SqlParserPos.ZERO, node, dataType); + } + + private SqlDataTypeSpec createRawDataTypeSpec(RawRelDataType type) { + final RawType rawType = type.getRawType(); + + SqlNode className = + SqlLiteral.createCharString( + rawType.getOriginatingClass().getName(), SqlParserPos.ZERO); + SqlNode serializer = + SqlLiteral.createCharString(rawType.getSerializerString(), SqlParserPos.ZERO); + + SqlTypeNameSpec rawSpec = new SqlRawTypeNameSpec(className, serializer, SqlParserPos.ZERO); + + return new SqlDataTypeSpec(rawSpec, null, type.isNullable(), SqlParserPos.ZERO); } /** Adopted from {@link org.apache.calcite.sql.validate.implicit.AbstractTypeCoercion}. */ 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 e07ec1372dac8..a85cf50c50ce0 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 @@ -18,7 +18,9 @@ package org.apache.flink.table.planner.runtime.batch.sql; +import org.apache.flink.table.annotation.DataTypeHint; import org.apache.flink.table.api.Table; +import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.planner.factories.utils.TestCollectionTableFactory; import org.apache.flink.table.planner.runtime.utils.BatchTestBase; import org.apache.flink.types.Row; @@ -99,6 +101,49 @@ void testUserDefinedTemporarySystemFunctionByUsingJar() throws Exception { testUserDefinedFunctionByUsingJar(functionDDL, dropFunctionDDL); } + @Test + void testOrderByScopeRawTypeCast() throws Exception { + final List sourceData = List.of(Row.of(1), Row.of(2), Row.of(3), Row.of(4), Row.of(5)); + TestCollectionTableFactory.reset(); + TestCollectionTableFactory.initData(sourceData); + + tEnv().executeSql("CREATE TABLE Source(i INT) WITH ('connector' = 'COLLECTION')"); + tEnv().executeSql("CREATE TABLE Sink(i INT) WITH ('connector' = 'COLLECTION')"); + + tEnv().createTemporarySystemFunction("CustomIntUdf", new CustomIntUdf()); + + tEnv().executeSql( + "INSERT INTO Sink" + + " SELECT i FROM Source" + + " ORDER BY CustomIntUdf(NULL)") + .await(); + + assertThat(TestCollectionTableFactory.getResult()).hasSize(5); + } + + @Test + void testHavingScopeRawTypeCast() throws Exception { + final List sourceData = List.of(Row.of(1), Row.of(2), Row.of(3), Row.of(4), Row.of(5)); + TestCollectionTableFactory.reset(); + TestCollectionTableFactory.initData(sourceData); + + tEnv().executeSql("CREATE TABLE Source(i INT) WITH ('connector' = 'COLLECTION')"); + tEnv().executeSql("CREATE TABLE Sink(i INT) WITH ('connector' = 'COLLECTION')"); + + tEnv().createTemporarySystemFunction("CustomIntUdf", new CustomIntUdf()); + + tEnv().executeSql( + "INSERT INTO Sink" + + " SELECT SUM(i) AS s FROM Source" + + " HAVING CustomIntUdf(NULL) = 0") + .await(); + + assertThat(TestCollectionTableFactory.getResult()) + .singleElement() + .asString() + .contains("15"); + } + private void testUserDefinedFunctionByUsingJar(String createFunctionDDL, String dropFunctionDDL) throws Exception { List sourceData = @@ -123,7 +168,7 @@ private void testUserDefinedFunctionByUsingJar(String createFunctionDDL, String Table t2 = tEnv().sqlQuery(query); t2.executeInsert("t2").await(); - List result = TestCollectionTableFactory.RESULT(); + List result = TestCollectionTableFactory.getResult(); List expected = Arrays.asList( Row.of(1, "jark"), @@ -139,4 +184,21 @@ private void testUserDefinedFunctionByUsingJar(String createFunctionDDL, String // delete the function tEnv().executeSql(dropFunctionDDL); } + + // ----- Test types / UDF ----- + + @DataTypeHint(value = "RAW", bridgedTo = CustomInt.class) + public static class CustomInt { + public Integer value; + + public CustomInt(Integer v) { + this.value = v; + } + } + + public static class CustomIntUdf extends ScalarFunction { + public Integer eval(CustomInt v) { + return 0; + } + } } From 434c80050e94bcbf8c902527896bb6bb7ee1baec Mon Sep 17 00:00:00 2001 From: Hao Li <1127478+lihaosky@users.noreply.github.com> Date: Tue, 18 Nov 2025 00:33:32 -0800 Subject: [PATCH 12/31] [FLINK-38686][doc] Add model table api documentation (#27243) (#27246) --- docs/content.zh/docs/dev/table/tableApi.md | 108 ++++++++++++++++++++ docs/content/docs/dev/table/tableApi.md | 109 +++++++++++++++++++++ 2 files changed, 217 insertions(+) diff --git a/docs/content.zh/docs/dev/table/tableApi.md b/docs/content.zh/docs/dev/table/tableApi.md index da78bf0fe7405..d46f01bef79d3 100644 --- a/docs/content.zh/docs/dev/table/tableApi.md +++ b/docs/content.zh/docs/dev/table/tableApi.md @@ -2735,6 +2735,114 @@ result = t.select(col('a'), col('c')) \ {{< query_state_warning_zh >}} +### 模型推理 + +{{< label Streaming >}} + +Table API 支持模型推理操作,允许你将机器学习模型直接集成到数据处理管道中。你可以使用特定的提供者创建模型,并使用它们对数据进行推理。 + +#### 创建和使用模型 + +使用 `ModelDescriptor` 创建模型,它指定提供者、输入/输出 schema 以及配置选项。创建后,你可以使用该模型对表进行预测。 + +{{< tabs "model-inference" >}} +{{< tab "Java" >}} + +```java +// 1. 设置本地环境 +EnvironmentSettings settings = EnvironmentSettings.inStreamingMode(); +TableEnvironment tEnv = TableEnvironment.create(settings); + +// 2. 从内存数据创建源表 +Table myTable = tEnv.fromValues( + ROW(FIELD("text", STRING())), + row("Hello"), + row("Machine Learning"), + row("Good morning") +); + +// 3. 创建模型 +tEnv.createModel( + "my_model", + ModelDescriptor.forProvider("openai") + .inputSchema(Schema.newBuilder().column("input", STRING()).build()) + .outputSchema(Schema.newBuilder().column("output", STRING()).build()) + .option("endpoint", "https://api.openai.com/v1/chat/completions") + .option("model", "gpt-4.1") + .option("system-prompt", "translate to chinese") + .option("api-key", "") + .build() +); + +Model model = tEnv.fromModel("my_model"); + +// 4. 使用模型把文本翻译成中文 +Table predictResult = model.predict(myTable, ColumnList.of("text")); + +// 5. 异步预测示例 +Table asyncPredictResult = model.predict( + myTable, + ColumnList.of("text"), + Map.of("async", "true") +); +``` + +{{< /tab >}} +{{< tab "Scala" >}} + +```scala +// 1. 设置本地环境 +val settings = EnvironmentSettings.inStreamingMode() +val tEnv = TableEnvironment.create(settings) + +// 2. 从内存数据创建源表 +val myTable: Table = tEnv.fromValues( + ROW(FIELD("text", STRING())), + row("Hello"), + row("Machine Learning"), + row("Good morning") +) + +// 3. 创建模型 +tEnv.createModel( + "my_model", + ModelDescriptor.forProvider("openai") + .inputSchema(Schema.newBuilder().column("input", STRING()).build()) + .outputSchema(Schema.newBuilder().column("output", STRING()).build()) + .option("endpoint", "https://api.openai.com/v1/chat/completions") + .option("model", "gpt-4.1") + .option("system-prompt", "translate to chinese") + .option("api-key", "") + .build() +) + +val model = tEnv.fromModel("my_model") + +// 4. 使用模型把文本翻译成中文 +val predictResult = model.predict(myTable, ColumnList.of("text")) + +// 5. 异步预测示例 +val asyncPredictResult = model.predict( + myTable, + ColumnList.of("text"), + Map("async" -> "true").asJava +) +``` + +{{< /tab >}} +{{< tab "Python" >}} + +```python +# 目前 Python Table API 尚不支持 +``` + +{{< /tab >}} +{{< /tabs >}} + +模型推理操作支持同步和异步预测模式 (需要底层接口 `ModelProvider`支持)。默认情况下, Planner使用异步预测。这可以通过允许并发请求来提高高延迟模型的吞吐量。 + +{{< top >}} + 数据类型 ---------- diff --git a/docs/content/docs/dev/table/tableApi.md b/docs/content/docs/dev/table/tableApi.md index d19cdcfc021ca..4b28838a3f237 100644 --- a/docs/content/docs/dev/table/tableApi.md +++ b/docs/content/docs/dev/table/tableApi.md @@ -2735,6 +2735,115 @@ result = t.select(col('a'), col('c')) \ {{< query_state_warning >}} +### Model Inference + +{{< label Streaming >}} + +The Table API supports model inference operations that allow you to integrate machine learning models directly into your data processing pipelines. You can create models with specific providers and use them to make inference on your data. + +#### Creating and Using Models + +Models are created using `ModelDescriptor` which specifies the provider, input/output schemas, and configuration options. Once created, you can use the model to make predictions on tables. + +{{< tabs "model-inference" >}} +{{< tab "Java" >}} + +```java +// 1. Set up the local environment +EnvironmentSettings settings = EnvironmentSettings.inStreamingMode(); +TableEnvironment tEnv = TableEnvironment.create(settings); + +// 2. Create a source table from in-memory data +Table myTable = tEnv.fromValues( + ROW(FIELD("text", STRING())), + row("Hello"), + row("Machine Learning"), + row("Good morning") +); + +// 3. Create model +tEnv.createModel( + "my_model", + ModelDescriptor.forProvider("openai") + .inputSchema(Schema.newBuilder().column("input", STRING()).build()) + .outputSchema(Schema.newBuilder().column("output", STRING()).build()) + .option("endpoint", "https://api.openai.com/v1/chat/completions") + .option("model", "gpt-4.1") + .option("system-prompt", "translate text to Chinese") + .option("api-key", "") + .build() +); + +Model model = tEnv.fromModel("my_model"); + +// 4. Use the model to translate text to Chinese +Table predictResult = model.predict(myTable, ColumnList.of("text")); + +// 5. Async prediction example +Table asyncPredictResult = model.predict( + myTable, + ColumnList.of("text"), + Map.of("async", "true") +); +``` + +{{< /tab >}} +{{< tab "Scala" >}} + +```scala +// 1. Set up the local environment +val settings = EnvironmentSettings.inStreamingMode() +val tEnv = TableEnvironment.create(settings) + +// 2. Create a source table from in-memory data +val myTable: Table = tEnv.fromValues( + ROW(FIELD("text", STRING())), + row("Hello"), + row("Machine Learning"), + row("Good morning") +) + +// 3. Create model +tEnv.createModel( + "my_model", + ModelDescriptor.forProvider("openai") + .inputSchema(Schema.newBuilder().column("input", STRING()).build()) + .outputSchema(Schema.newBuilder().column("output", STRING()).build()) + .option("endpoint", "https://api.openai.com/v1/chat/completions") + .option("model", "gpt-4.1") + .option("system-prompt", "translate to chinese") + .option("api-key", "") + .build() +) + +val model = tEnv.fromModel("my_model") + +// 4. Use the model to translate text to Chinese +val predictResult = model.predict(myTable, ColumnList.of("text")) + +// 5. Async prediction example +val asyncPredictResult = model.predict( + myTable, + ColumnList.of("text"), + Map("async" -> "true").asJava +) +``` + +{{< /tab >}} +{{< tab "Python" >}} + +```python +# Not yet supported in Python Table API +``` + +{{< /tab >}} +{{< /tabs >}} + +Model inference supports both synchronous and asynchronous prediction modes (when supported by the underlying `ModelProvider` interface). +By default, the planner uses asynchronous mode to maximize throughput for high-latency models by processing multiple requests concurrently. + +{{< top >}} + Data Types ---------- From ef31ad4086fd6ef1d67fa41c4e9c245d5e54299e Mon Sep 17 00:00:00 2001 From: Xuyang Date: Thu, 20 Nov 2025 17:23:31 +0800 Subject: [PATCH 13/31] [FLINK-38695][table-planner] Fix wrong metric about left cache request count in DeltaJoinCache (#27253) --- .../join/deltajoin/DeltaJoinCache.java | 17 +- .../join/deltajoin/DeltaJoinCacheTest.java | 157 ++++++++++++++++++ 2 files changed, 166 insertions(+), 8 deletions(-) create mode 100644 flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinCacheTest.java diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinCache.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinCache.java index ffaca5736bd70..9a10edd26261c 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinCache.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinCache.java @@ -41,16 +41,17 @@ *

Note: This cache is not thread-safe although its inner {@link Cache} is thread-safe. */ @NotThreadSafe +@VisibleForTesting public class DeltaJoinCache { - private static final String LEFT_CACHE_METRIC_PREFIX = "deltaJoin.leftCache."; - private static final String RIGHT_CACHE_METRIC_PREFIX = "deltaJoin.rightCache."; + protected static final String LEFT_CACHE_METRIC_PREFIX = "deltaJoin.leftCache."; + protected static final String RIGHT_CACHE_METRIC_PREFIX = "deltaJoin.rightCache."; - private static final String METRIC_HIT_RATE = "hitRate"; - private static final String METRIC_REQUEST_COUNT = "requestCount"; - private static final String METRIC_HIT_COUNT = "hitCount"; - private static final String METRIC_KEY_SIZE = "keySize"; - private static final String METRIC_TOTAL_NON_EMPTY_VALUE_SIZE = "totalNonEmptyValues"; + protected static final String METRIC_HIT_RATE = "hitRate"; + protected static final String METRIC_REQUEST_COUNT = "requestCount"; + protected static final String METRIC_HIT_COUNT = "hitCount"; + protected static final String METRIC_KEY_SIZE = "keySize"; + protected static final String METRIC_TOTAL_NON_EMPTY_VALUE_SIZE = "totalNonEmptyValues"; // use LinkedHashMap to keep order private final Cache> leftCache; @@ -87,7 +88,7 @@ public void registerMetrics(MetricGroup metricGroup) { : Long.valueOf(leftHitCount.get()).doubleValue() / leftRequestCount.get()); metricGroup.>gauge( - LEFT_CACHE_METRIC_PREFIX + METRIC_REQUEST_COUNT, rightRequestCount::get); + LEFT_CACHE_METRIC_PREFIX + METRIC_REQUEST_COUNT, leftRequestCount::get); metricGroup.>gauge( LEFT_CACHE_METRIC_PREFIX + METRIC_HIT_COUNT, leftHitCount::get); metricGroup.>gauge( diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinCacheTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinCacheTest.java new file mode 100644 index 0000000000000..7c23e794ca58c --- /dev/null +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinCacheTest.java @@ -0,0 +1,157 @@ +/* + * 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.runtime.operators.join.deltajoin; + +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Metric; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; +import org.apache.flink.table.data.RowData; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; +import org.apache.flink.util.function.TriConsumer; + +import org.apache.flink.shaded.guava33.com.google.common.collect.Maps; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.function.BiConsumer; + +import static org.apache.flink.table.runtime.operators.join.deltajoin.DeltaJoinCache.LEFT_CACHE_METRIC_PREFIX; +import static org.apache.flink.table.runtime.operators.join.deltajoin.DeltaJoinCache.METRIC_HIT_COUNT; +import static org.apache.flink.table.runtime.operators.join.deltajoin.DeltaJoinCache.METRIC_HIT_RATE; +import static org.apache.flink.table.runtime.operators.join.deltajoin.DeltaJoinCache.METRIC_KEY_SIZE; +import static org.apache.flink.table.runtime.operators.join.deltajoin.DeltaJoinCache.METRIC_REQUEST_COUNT; +import static org.apache.flink.table.runtime.operators.join.deltajoin.DeltaJoinCache.METRIC_TOTAL_NON_EMPTY_VALUE_SIZE; +import static org.apache.flink.table.runtime.operators.join.deltajoin.DeltaJoinCache.RIGHT_CACHE_METRIC_PREFIX; +import static org.apache.flink.table.runtime.util.StreamRecordUtils.row; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link DeltaJoinCache}. */ +@ExtendWith(ParameterizedTestExtension.class) +class DeltaJoinCacheTest { + + private static final Long LEFT_CACHE_SIZE = 3L; + private static final Long RIGHT_CACHE_SIZE = 2L; + + @Parameters(name = "testRightCache = {0}") + private static List parameters() { + return Arrays.asList(false, true); + } + + @Parameter private boolean testRightCache; + + private DeltaJoinCache cache; + private Runnable requestCacheFunc; + private Runnable hitCacheFunc; + private BiConsumer> buildCacheFunc; + private TriConsumer upsertCacheFunc; + + @BeforeEach + void before() { + cache = new DeltaJoinCache(LEFT_CACHE_SIZE, RIGHT_CACHE_SIZE); + + requestCacheFunc = + () -> { + if (testRightCache) { + cache.requestRightCache(); + } else { + cache.requestLeftCache(); + } + }; + hitCacheFunc = + () -> { + if (testRightCache) { + cache.hitRightCache(); + } else { + cache.hitLeftCache(); + } + }; + buildCacheFunc = (key, ukDataMap) -> cache.buildCache(key, ukDataMap, testRightCache); + upsertCacheFunc = (key, uk, data) -> cache.upsertCache(key, uk, data, testRightCache); + } + + @TestTemplate + void testReportMetrics() { + Map allMetrics = new HashMap<>(); + cache.registerMetrics( + new UnregisteredMetricGroups.UnregisteredOperatorMetricGroup() { + @Override + protected void addMetric(String name, Metric metric) { + allMetrics.put(name, metric); + super.addMetric(name, metric); + } + }); + + assertReportMetricsInternal(allMetrics, 0, 0, 0.0, 0, 0); + requestCacheFunc.run(); + assertReportMetricsInternal(allMetrics, 1, 0, 0.0, 0, 0); + hitCacheFunc.run(); + assertReportMetricsInternal(allMetrics, 1, 1, 1.0, 0, 0); + requestCacheFunc.run(); + assertReportMetricsInternal(allMetrics, 2, 1, 0.5, 0, 0); + + buildCacheFunc.accept(row("ck1"), Maps.newLinkedHashMap()); + assertReportMetricsInternal(allMetrics, 2, 1, 0.5, 1, 0); + buildCacheFunc.accept( + row("ck2"), + Maps.newLinkedHashMap(Map.of(row("pk1"), 1, row("pk2"), 2, row("pk3"), 3))); + assertReportMetricsInternal(allMetrics, 2, 1, 0.5, 2, 3); + upsertCacheFunc.accept(row("ck1"), row("pk4"), 4); + assertReportMetricsInternal(allMetrics, 2, 1, 0.5, 2, 4); + } + + @SuppressWarnings("unchecked") + private void assertReportMetricsInternal( + Map actualAllMetrics, + long expectedRequestCount, + long expectedHitCount, + double expectedHitRate, + long expectedKeySize, + long expectedNonEmptyValueSize) { + String prefix = testRightCache ? RIGHT_CACHE_METRIC_PREFIX : LEFT_CACHE_METRIC_PREFIX; + + String hitRate = prefix + METRIC_HIT_RATE; + assertThat(((Gauge) actualAllMetrics.get(hitRate)).getValue()) + .isEqualTo(expectedHitRate); + + String requestCount = prefix + METRIC_REQUEST_COUNT; + assertThat(((Gauge) actualAllMetrics.get(requestCount)).getValue()) + .isEqualTo(expectedRequestCount); + + String hitCount = prefix + METRIC_HIT_COUNT; + assertThat(((Gauge) actualAllMetrics.get(hitCount)).getValue()) + .isEqualTo(expectedHitCount); + + String keySize = prefix + METRIC_KEY_SIZE; + assertThat(((Gauge) actualAllMetrics.get(keySize)).getValue()) + .isEqualTo(expectedKeySize); + + String totalNonEmptyValueSize = prefix + METRIC_TOTAL_NON_EMPTY_VALUE_SIZE; + assertThat(((Gauge) actualAllMetrics.get(totalNonEmptyValueSize)).getValue()) + .isEqualTo(expectedNonEmptyValueSize); + } +} From 4fd5bea812ee5726c0c4ee5e27fda320273ebb6e Mon Sep 17 00:00:00 2001 From: Yuepeng Pan Date: Mon, 3 Nov 2025 23:54:45 +0800 Subject: [PATCH 14/31] [FLINK-36746][core] Fix the deadlock bug in SerializedThrowable Co-authored-by: raoraoxiong Co-authored-by: Yun Tang This closes #27186 --- .../flink/util/SerializedThrowable.java | 38 +++++++---- .../runtime/util/SerializedThrowableTest.java | 68 +++++++++++++++++++ 2 files changed, 93 insertions(+), 13 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/util/SerializedThrowable.java b/flink-core/src/main/java/org/apache/flink/util/SerializedThrowable.java index ae1ea5e14e159..7937e2de8a68a 100644 --- a/flink-core/src/main/java/org/apache/flink/util/SerializedThrowable.java +++ b/flink-core/src/main/java/org/apache/flink/util/SerializedThrowable.java @@ -68,10 +68,14 @@ private SerializedThrowable(Throwable exception, Set alreadySeen) { if (!(exception instanceof SerializedThrowable)) { // serialize and memoize the original message byte[] serialized; - try { - serialized = InstantiationUtil.serializeObject(exception); - } catch (Throwable t) { - serialized = null; + // introduce the synchronization here to avoid deadlock of multi thread serializing + // exceptions + synchronized (SerializedThrowable.class) { + try { + serialized = InstantiationUtil.serializeObject(exception); + } catch (Throwable t) { + serialized = null; + } } this.serializedException = serialized; this.cachedException = new WeakReference<>(exception); @@ -94,7 +98,7 @@ private SerializedThrowable(Throwable exception, Set alreadySeen) { } } // mimic suppressed exceptions - addAllSuppressed(exception.getSuppressed()); + this.addAllSuppressed(exception.getSuppressed(), alreadySeen); } else { // copy from that serialized throwable SerializedThrowable other = (SerializedThrowable) exception; @@ -104,7 +108,7 @@ private SerializedThrowable(Throwable exception, Set alreadySeen) { this.cachedException = other.cachedException; this.setStackTrace(other.getStackTrace()); this.initCause(other.getCause()); - this.addAllSuppressed(other.getSuppressed()); + this.addAllSuppressed(other.getSuppressed(), alreadySeen); } } @@ -141,15 +145,23 @@ public String getFullStringifiedStackTrace() { return fullStringifiedStackTrace; } - private void addAllSuppressed(Throwable[] suppressed) { + /** + * Add all suppressed exceptions to this exception. + * + * @param suppressed The suppressed exceptions to add. + * @param alreadySeen The set of exceptions that have already been seen. + */ + private void addAllSuppressed(Throwable[] suppressed, Set alreadySeen) { for (Throwable s : suppressed) { - SerializedThrowable serializedThrowable; - if (s instanceof SerializedThrowable) { - serializedThrowable = (SerializedThrowable) s; - } else { - serializedThrowable = new SerializedThrowable(s); + if (alreadySeen.add(s)) { + SerializedThrowable serializedThrowable; + if (s instanceof SerializedThrowable) { + serializedThrowable = (SerializedThrowable) s; + } else { + serializedThrowable = new SerializedThrowable(s); + } + this.addSuppressed(serializedThrowable); } - this.addSuppressed(serializedThrowable); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java index 2f81a91fd2076..64a9340937eed 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java @@ -19,12 +19,22 @@ package org.apache.flink.runtime.util; import org.apache.flink.core.testutils.CommonTestUtils; +import org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException; import org.apache.flink.testutils.ClassLoaderUtils; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.SerializedThrowable; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -179,4 +189,62 @@ void testCopySuppressed() { .isInstanceOf(SerializedThrowable.class) .hasMessage("java.lang.Exception: suppressed"); } + + @Test + void testCyclicSuppressedThrowableSerialized() { + SerializedThrowable serializedThrowable = new SerializedThrowable(mockThrowable()); + assertThat(serializedThrowable).isNotNull(); + } + + @Test + @Timeout(value = 5, unit = TimeUnit.SECONDS) + void testCyclicSuppressedThrowableConcurrentSerialized() throws InterruptedException { + Throwable throwable = mockThrowable(); + int threadNum = 16; + CountDownLatch countDownLatch = new CountDownLatch(threadNum); + List threads = new ArrayList<>(); + for (int i = 0; i < threadNum; i++) { + String threadName = "thread-" + i; + Thread t = createThread(countDownLatch, throwable, threadName); + t.start(); + countDownLatch.countDown(); + threads.add(t); + } + for (Thread thread : threads) { + thread.join(); + } + } + + private static Thread createThread( + CountDownLatch countDownLatch, Throwable throwable, String threadName) { + Thread t = + new Thread( + () -> { + try { + countDownLatch.await(); + SerializedThrowable serializedThrowable = + new SerializedThrowable(throwable); + assertThat(serializedThrowable).isNotNull(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + t.setName(threadName); + return t; + } + + private static Throwable mockThrowable() { + SocketAddress remoteAddr = new InetSocketAddress(80); + RemoteTransportException remoteTransportException = + new RemoteTransportException( + "Connection unexpectedly closed by remote task manager '" + + remoteAddr + + "'. " + + "This might indicate that the remote task manager was lost.", + remoteAddr, + new IOException("connection reset by peer.")); + RuntimeException runtimeException = new RuntimeException(remoteTransportException); + remoteTransportException.addSuppressed(runtimeException); + return remoteTransportException; + } } From e666847b119b3fab693d636cc2078b23706cda7c Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Thu, 20 Nov 2025 09:45:19 +0800 Subject: [PATCH 15/31] [FLINK-38709][table][python] Fix ScalarFunctionSplitter to allow PythonFunction & AsyncFunction work when taking the recursive field of composite type as input This closes #27259. --- .../rules/logical/RemoteCalcSplitRule.scala | 15 +- .../rules/logical/AsyncCalcSplitRuleTest.java | 9 +- .../logical/AsyncCorrelateSplitRuleTest.java | 9 +- .../rules/logical/AsyncCalcSplitRuleTest.xml | 166 ++++++++++-------- .../logical/AsyncCorrelateSplitRuleTest.xml | 83 +++++---- 5 files changed, 174 insertions(+), 108 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/RemoteCalcSplitRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/RemoteCalcSplitRule.scala index ff60809cea680..26baa44f43d82 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/RemoteCalcSplitRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/RemoteCalcSplitRule.scala @@ -434,6 +434,8 @@ class ScalarFunctionSplitter( private var fieldsRexCall: Map[Int, Int] = Map[Int, Int]() + private val extractedRexNodeRefs: mutable.HashSet[RexNode] = mutable.HashSet[RexNode]() + override def visitCall(call: RexCall): RexNode = { if (needConvert(call)) { getExtractedRexNode(call) @@ -454,7 +456,9 @@ class ScalarFunctionSplitter( new RexInputRef(field.getIndex, field.getType) case _ => val newFieldAccess = - rexBuilder.makeFieldAccess(expr.accept(this), fieldAccess.getField.getIndex) + rexBuilder.makeFieldAccess( + convertInputRefToLocalRefIfNecessary(expr.accept(this)), + fieldAccess.getField.getIndex) getExtractedRexNode(newFieldAccess) } } else { @@ -468,9 +472,18 @@ class ScalarFunctionSplitter( override def visitNode(rexNode: RexNode): RexNode = rexNode + private def convertInputRefToLocalRefIfNecessary(node: RexNode): RexNode = { + node match { + case inputRef: RexInputRef if extractedRexNodeRefs.contains(node) => + new RexLocalRef(inputRef.getIndex, node.getType) + case _ => node + } + } + private def getExtractedRexNode(node: RexNode): RexNode = { val newNode = new RexInputRef(extractedFunctionOffset + extractedRexNodes.length, node.getType) extractedRexNodes.append(node) + extractedRexNodeRefs.add(newNode) newNode } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRuleTest.java index 444c17fee7e49..53ba394f5f07d 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRuleTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRuleTest.java @@ -62,7 +62,8 @@ public void setup() { + " a int,\n" + " b bigint,\n" + " c string,\n" - + " d ARRAY\n" + + " d ARRAY,\n" + + " e ROW, g string>" + ") WITH (\n" + " 'connector' = 'test-simple-table-source'\n" + ") ;"); @@ -182,6 +183,12 @@ public void testFieldAccessAfter() { util.verifyRelPlan(sqlQuery); } + @Test + public void testCompositeFieldAsInput() { + String sqlQuery = "SELECT func1(e.f.h) from MyTable"; + util.verifyRelPlan(sqlQuery); + } + @Test public void testFieldOperand() { String sqlQuery = "SELECT func1(func5(a).f0) from MyTable"; diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRuleTest.java index b98403a28466a..862aedb421b24 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRuleTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRuleTest.java @@ -62,7 +62,8 @@ public void setup() { + " a int,\n" + " b bigint,\n" + " c string,\n" - + " d ARRAY\n" + + " d ARRAY,\n" + + " e ROW, g string>\n" + ") WITH (\n" + " 'connector' = 'test-simple-table-source'\n" + ") ;"); @@ -110,6 +111,12 @@ public void testCorrelateWithCast() { util.verifyRelPlan(sqlQuery); } + @Test + public void testCorrelateWithCompositeFieldAsInput() { + String sqlQuery = "select * FROM MyTable, LATERAL TABLE(asyncTableFunc(e.f.h))"; + util.verifyRelPlan(sqlQuery); + } + /** Test function. */ public static class AsyncFunc extends AsyncTableFunction { diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRuleTest.xml index fa4f17124edbc..726b02847c8a5 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRuleTest.xml @@ -34,7 +34,25 @@ AsyncCalc(select=[a, func3($f1) AS EXPR$1]) +- GroupAggregate(groupBy=[a], select=[a, COUNT(*) AS $f1]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) +]]> + + + + + + + + + + + @@ -52,7 +70,7 @@ LogicalProject(EXPR$0=[func5($0).f0]) @@ -71,7 +89,7 @@ LogicalProject(EXPR$0=[func1(func5($0).f0)]) AsyncCalc(select=[func1(f0) AS EXPR$0]) +- Calc(select=[f0.f0 AS f0]) +- AsyncCalc(select=[func5(a) AS f0]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) ]]> @@ -88,7 +106,7 @@ LogicalProject(a=[$0], EXPR$1=[func1($0)]) @@ -107,7 +125,7 @@ LogicalProject(EXPR$0=[func1($0)], EXPR$1=[func2($0)], EXPR$2=[func1($0)], EXPR$ Calc(select=[f0 AS EXPR$0, f00 AS EXPR$1, f0 AS EXPR$2, f00 AS EXPR$3]) +- AsyncCalc(select=[f0, func2(a) AS f00]) +- AsyncCalc(select=[a, func1(a) AS f0]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) ]]> @@ -118,9 +136,9 @@ Calc(select=[f0 AS EXPR$0, f00 AS EXPR$1, f0 AS EXPR$2, f00 AS EXPR$3]) (func6($0, $4), 10))], joinType=[inner]) ++- LogicalJoin(condition=[AND(=($0, $5), >(func6($0, $5), 10))], joinType=[inner]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -159,7 +177,7 @@ Calc(select=[a], where=[>(f0, 10)]) +- Join(joinType=[InnerJoin], where=[=(a, a2)], select=[a, a2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a]) - : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) +- Exchange(distribution=[hash[a2]]) +- Calc(select=[a2]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a2, b2, c2, d2]) @@ -174,7 +192,7 @@ Calc(select=[a], where=[>(f0, 10)]) @@ -186,7 +204,7 @@ Calc(select=[a]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a], where=[REGEXP(f0, 'val (2|3)')]) : +- AsyncCalc(select=[a, func2(a) AS f0]) - : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) +- Exchange(distribution=[hash[a2]]) +- Calc(select=[a2], where=[REGEXP(f0, 'val (2|3)')]) +- AsyncCalc(select=[a2, func2(a2) AS f0]) @@ -201,8 +219,8 @@ Calc(select=[a]) (func6($0, $4), 10)]) - +- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) ++- LogicalFilter(condition=[>(func6($0, $5), 10)]) + +- LogicalJoin(condition=[=($0, $5)], joinType=[inner]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -214,7 +232,7 @@ Calc(select=[a], where=[>(f0, 10)]) +- Join(joinType=[InnerJoin], where=[=(a, a2)], select=[a, a2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a]) - : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) +- Exchange(distribution=[hash[a2]]) +- Calc(select=[a2]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a2, b2, c2, d2]) @@ -228,7 +246,7 @@ Calc(select=[a], where=[>(f0, 10)]) @@ -239,7 +257,7 @@ AsyncCalc(select=[func1(a) AS EXPR$0]) +- Join(joinType=[InnerJoin], where=[=(a, a2)], select=[a, a2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a]) - : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) +- Exchange(distribution=[hash[a2]]) +- Calc(select=[a2]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a2, b2, c2, d2]) @@ -270,8 +288,8 @@ AsyncCalc(select=[func1(1) AS EXPR$0]) (func6($0, $4), 10))]) - +- LogicalJoin(condition=[=($0, $4)], joinType=[left]) ++- LogicalFilter(condition=[AND(=($0, $5), >(func6($0, $5), 10))]) + +- LogicalJoin(condition=[=($0, $5)], joinType=[left]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -283,7 +301,7 @@ Calc(select=[a], where=[>(f0, 10)]) +- Join(joinType=[InnerJoin], where=[=(a, a2)], select=[a, a2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a]) - : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) +- Exchange(distribution=[hash[a2]]) +- Calc(select=[a2]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a2, b2, c2, d2]) @@ -296,10 +314,10 @@ Calc(select=[a], where=[>(f0, 10)]) @@ -337,7 +355,7 @@ Join(joinType=[LeftOuterJoin], where=[=(a, a2)], select=[a, a2], leftInputSpec=[ :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a], where=[REGEXP(f0, 'string (2|3)')]) : +- AsyncCalc(select=[a, func2(a) AS f0]) -: +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) +- Exchange(distribution=[hash[a2]]) +- Calc(select=[a2]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a2, b2, c2, d2]) @@ -351,8 +369,8 @@ Join(joinType=[LeftOuterJoin], where=[=(a, a2)], select=[a, a2], leftInputSpec=[ (func6($0, $4), 10)]) - +- LogicalJoin(condition=[=($0, $4)], joinType=[left]) ++- LogicalFilter(condition=[>(func6($0, $5), 10)]) + +- LogicalJoin(condition=[=($0, $5)], joinType=[left]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -364,7 +382,7 @@ Calc(select=[a], where=[>(f0, 10)]) +- Join(joinType=[LeftOuterJoin], where=[=(a, a2)], select=[a, a2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a]) - : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) +- Exchange(distribution=[hash[a2]]) +- Calc(select=[a2]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a2, b2, c2, d2]) @@ -378,7 +396,7 @@ Calc(select=[a], where=[>(f0, 10)]) (func1($4), 10))], joinType=[left]) ++- LogicalJoin(condition=[AND(=($0, $5), >(func1($5), 10))], joinType=[left]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -389,7 +407,7 @@ Calc(select=[a]) +- Join(joinType=[LeftOuterJoin], where=[=(a, a2)], select=[a, a2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a]) - : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) +- Exchange(distribution=[hash[a2]]) +- Calc(select=[a2], where=[>(f0, 10)]) +- AsyncCalc(select=[a2, func1(a2) AS f0]) @@ -404,8 +422,8 @@ Calc(select=[a]) (func1($4), 10)]) - +- LogicalJoin(condition=[=($0, $4)], joinType=[left]) ++- LogicalFilter(condition=[>(func1($5), 10)]) + +- LogicalJoin(condition=[=($0, $5)], joinType=[left]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -417,7 +435,7 @@ Calc(select=[a], where=[>(f0, 10)]) +- Join(joinType=[LeftOuterJoin], where=[=(a, a2)], select=[a, a2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a]) - : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) +- Exchange(distribution=[hash[a2]]) +- Calc(select=[a2]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a2, b2, c2, d2]) @@ -438,7 +456,7 @@ LogicalProject(EXPR$0=[_UTF-16LE'foo'], EXPR$1=[func1($0)]) @@ -457,7 +475,7 @@ LogicalProject(EXPR$0=[func1(func1(func1($0)))]) AsyncCalc(select=[func1(f0) AS EXPR$0]) +- AsyncCalc(select=[func1(f0) AS f0]) +- AsyncCalc(select=[func1(a) AS f0]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) ]]> @@ -493,7 +511,7 @@ LogicalProject(EXPR$0=[CONCAT(func2($0), _UTF-16LE'foo')]) @@ -513,7 +531,7 @@ LogicalProject(blah=[$0]) @@ -524,8 +542,8 @@ Calc(select=[f0 AS blah], where=[REGEXP(f0, 'string (2|3)')]) (func6($0, $4), 10))]) - +- LogicalJoin(condition=[=($0, $4)], joinType=[right]) ++- LogicalFilter(condition=[AND(=($0, $5), >(func6($0, $5), 10))]) + +- LogicalJoin(condition=[=($0, $5)], joinType=[right]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -537,7 +555,7 @@ Calc(select=[a], where=[>(f0, 10)]) +- Join(joinType=[InnerJoin], where=[=(a, a2)], select=[a, a2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a]) - : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) +- Exchange(distribution=[hash[a2]]) +- Calc(select=[a2]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a2, b2, c2, d2]) @@ -550,10 +568,10 @@ Calc(select=[a], where=[>(f0, 10)]) (func6($0, $4), 10)]) - +- LogicalJoin(condition=[=($0, $4)], joinType=[right]) ++- LogicalFilter(condition=[>(func6($0, $5), 10)]) + +- LogicalJoin(condition=[=($0, $5)], joinType=[right]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -592,7 +610,7 @@ Calc(select=[a], where=[>(f0, 10)]) +- Join(joinType=[RightOuterJoin], where=[=(a, a2)], select=[a, a2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a]) - : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) +- Exchange(distribution=[hash[a2]]) +- Calc(select=[a2]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a2, b2, c2, d2]) @@ -606,7 +624,7 @@ Calc(select=[a], where=[>(f0, 10)]) (func1($0), 10))], joinType=[right]) ++- LogicalJoin(condition=[AND(=($0, $5), >(func1($0), 10))], joinType=[right]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -618,7 +636,7 @@ Calc(select=[a]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a], where=[>(f0, 10)]) : +- AsyncCalc(select=[a, func1(a) AS f0]) - : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) +- Exchange(distribution=[hash[a2]]) +- Calc(select=[a2]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a2, b2, c2, d2]) @@ -633,7 +651,7 @@ Calc(select=[a]) (func1($0), 10)]) - +- LogicalJoin(condition=[=($0, $4)], joinType=[right]) + +- LogicalJoin(condition=[=($0, $5)], joinType=[right]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -645,7 +663,7 @@ Calc(select=[a], where=[>(f0, 10)]) +- Join(joinType=[RightOuterJoin], where=[=(a, a2)], select=[a, a2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a]) - : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) +- Exchange(distribution=[hash[a2]]) +- Calc(select=[a2]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a2, b2, c2, d2]) @@ -665,7 +683,7 @@ LogicalProject(EXPR$0=[func4($3)]) @@ -682,7 +700,7 @@ LogicalProject(EXPR$0=[func1($0)]) @@ -703,7 +721,7 @@ AsyncCalc(select=[f0 AS EXPR$0, func1(f1) AS EXPR$1, f2 AS EXPR$2]) +- AsyncCalc(select=[f2, f1, func1(f0) AS f0]) +- Calc(select=[f0, f0 AS f1, f0 AS f2]) +- AsyncCalc(select=[func1(a) AS f0]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) ]]> @@ -721,7 +739,7 @@ LogicalProject(EXPR$0=[func1($0)], EXPR$1=[func1($0)]) @@ -740,7 +758,7 @@ LogicalProject(a=[$0]) @@ -759,7 +777,7 @@ LogicalProject(EXPR$0=[func2($0)]) @@ -779,7 +797,7 @@ LogicalProject(blah=[$0]) @@ -798,7 +816,7 @@ LogicalProject(a=[$0]) =(f0, 12)]) +- AsyncCalc(select=[a, func1(a) AS f0]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) ]]> @@ -822,7 +840,7 @@ AsyncCalc(select=[func1(a) AS EXPR$0]) +- Join(joinType=[LeftAntiJoin], where=[OR(IS NULL(a), IS NULL(a2), =(a, a2))], select=[a], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[single]) : +- Calc(select=[a]) - : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e]) +- Exchange(distribution=[single]) +- Calc(select=[a2]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable2]], fields=[a2, b2, c2, d2]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRuleTest.xml index dcbe743d2c2c7..2cdbfbaf162f8 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRuleTest.xml @@ -22,7 +22,7 @@ limitations under the License. @@ -43,7 +43,7 @@ Calc(select=[a, b, c, d, EXPR$0]) @@ -64,7 +64,7 @@ Calc(select=[a, b, c, d, EXPR$0]) @@ -86,7 +86,7 @@ Calc(select=[a, b, c, d, EXPR$0]) + + + + + + + + + + + @@ -107,7 +128,7 @@ Calc(select=[a, b, c, d, EXPR$0]) @@ -128,7 +149,7 @@ Calc(select=[a, b, c, d, EXPR$0]) From bf9cc5d3dfa03a56110c8b4de9f57f18aacf9753 Mon Sep 17 00:00:00 2001 From: Sergey Nuyanzin Date: Tue, 25 Nov 2025 07:53:46 +0100 Subject: [PATCH 16/31] [FLINK-38700][python] Fix kubernetes pyflink application test --------- Co-authored-by: Dian Fu --- .../test-scripts/test_kubernetes_pyflink_application.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flink-end-to-end-tests/test-scripts/test_kubernetes_pyflink_application.sh b/flink-end-to-end-tests/test-scripts/test_kubernetes_pyflink_application.sh index 389ee33b79277..da93a77c71fbc 100755 --- a/flink-end-to-end-tests/test-scripts/test_kubernetes_pyflink_application.sh +++ b/flink-end-to-end-tests/test-scripts/test_kubernetes_pyflink_application.sh @@ -80,7 +80,9 @@ cp "${FLINK_PYTHON_DIR}/pyproject.toml" $PYFLINK_DOCKER_DIR/ if [[ -d "dist" ]]; then rm -Rf dist; fi cd ${PYFLINK_DOCKER_DIR} echo "FROM ${PURE_FLINK_IMAGE_NAME}" >> Dockerfile +echo "USER root" >> Dockerfile echo "RUN apt-get update && apt-get install build-essential -y" >> Dockerfile +echo "USER flink" >> Dockerfile echo "COPY lint-python.sh /tmp/lint-python.sh" >> Dockerfile echo "COPY pyproject.toml /tmp/pyproject.toml" >> Dockerfile echo "RUN bash /tmp/lint-python.sh -s basic" >> Dockerfile From 2910d729f39d8e706900102707f8850d3a71f112 Mon Sep 17 00:00:00 2001 From: Mate Czagany <4469996+mateczagany@users.noreply.github.com> Date: Tue, 25 Nov 2025 12:51:30 +0100 Subject: [PATCH 17/31] [BP-2.2][FLINK-38711][build] Set maven-shade-plugin logs to DEBUG --- flink-models/flink-model-openai/pom.xml | 1 + .../flink-model-openai/src/main/resources/META-INF/NOTICE | 1 + tools/ci/maven-utils.sh | 2 +- 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/flink-models/flink-model-openai/pom.xml b/flink-models/flink-model-openai/pom.xml index f251b2e0e266f..f68d82db40d71 100644 --- a/flink-models/flink-model-openai/pom.xml +++ b/flink-models/flink-model-openai/pom.xml @@ -76,6 +76,7 @@ under the License. com.knuddels jtokkit 1.1.0 + ${flink.markBundledAsOptional} diff --git a/flink-models/flink-model-openai/src/main/resources/META-INF/NOTICE b/flink-models/flink-model-openai/src/main/resources/META-INF/NOTICE index c8908682f8865..292cdcd480037 100644 --- a/flink-models/flink-model-openai/src/main/resources/META-INF/NOTICE +++ b/flink-models/flink-model-openai/src/main/resources/META-INF/NOTICE @@ -13,6 +13,7 @@ This project bundles the following dependencies under the Apache Software Licens - com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.18.2 - com.fasterxml.jackson.module:jackson-module-kotlin:2.18.2 - com.google.errorprone:error_prone_annotations:2.33.0 +- com.knuddels:jtokkit:1.1.0 - com.openai:openai-java:1.6.1 - com.openai:openai-java-client-okhttp:1.6.1 - com.openai:openai-java-core:1.6.1 diff --git a/tools/ci/maven-utils.sh b/tools/ci/maven-utils.sh index ab535af4320b7..ac0ae9df1af1c 100755 --- a/tools/ci/maven-utils.sh +++ b/tools/ci/maven-utils.sh @@ -73,7 +73,7 @@ export MVN_GLOBAL_OPTIONS_WITHOUT_MIRROR="$MAVEN_ARGS " # see https://developercommunity.visualstudio.com/content/problem/851041/microsoft-hosted-agents-run-into-maven-central-tim.html MVN_GLOBAL_OPTIONS_WITHOUT_MIRROR+="-Dmaven.wagon.http.pool=false " # logging -MVN_GLOBAL_OPTIONS_WITHOUT_MIRROR+="-Dorg.slf4j.simpleLogger.showDateTime=true -Dorg.slf4j.simpleLogger.dateTimeFormat=HH:mm:ss.SSS -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn " +MVN_GLOBAL_OPTIONS_WITHOUT_MIRROR+="-Dorg.slf4j.simpleLogger.showDateTime=true -Dorg.slf4j.simpleLogger.dateTimeFormat=HH:mm:ss.SSS -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn -Dorg.slf4j.simpleLogger.log.org.apache.maven.plugins.shade=DEBUG " # suppress snapshot updates MVN_GLOBAL_OPTIONS_WITHOUT_MIRROR+="--no-snapshot-updates " # enable non-interactive batch mode From fefb74162c6bbaae89f35439f3a42c3e92b4b462 Mon Sep 17 00:00:00 2001 From: Mate Czagany <4469996+mateczagany@users.noreply.github.com> Date: Wed, 26 Nov 2025 09:55:49 +0100 Subject: [PATCH 18/31] [hotfix] Fix flink-model-openai NOTICE for jtokkit --- .../flink-model-openai/src/main/resources/META-INF/NOTICE | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/flink-models/flink-model-openai/src/main/resources/META-INF/NOTICE b/flink-models/flink-model-openai/src/main/resources/META-INF/NOTICE index 292cdcd480037..6273d7f94f1e1 100644 --- a/flink-models/flink-model-openai/src/main/resources/META-INF/NOTICE +++ b/flink-models/flink-model-openai/src/main/resources/META-INF/NOTICE @@ -13,7 +13,6 @@ This project bundles the following dependencies under the Apache Software Licens - com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.18.2 - com.fasterxml.jackson.module:jackson-module-kotlin:2.18.2 - com.google.errorprone:error_prone_annotations:2.33.0 -- com.knuddels:jtokkit:1.1.0 - com.openai:openai-java:1.6.1 - com.openai:openai-java-client-okhttp:1.6.1 - com.openai:openai-java-core:1.6.1 @@ -30,3 +29,7 @@ This project bundles the following dependencies under the Apache Software Licens - org.jetbrains.kotlin:kotlin-stdlib-common:1.9.10 - org.jetbrains.kotlin:kotlin-stdlib-jdk7:1.9.10 - org.jetbrains.kotlin:kotlin-stdlib-jdk8:1.9.10 + +This project bundles the following dependencies under the MIT (https://opensource.org/licenses/MIT) + +- com.knuddels:jtokkit:1.1.0 From e4c197d3bc2c2c2c87ccb987d7937c8c2e3cf2c1 Mon Sep 17 00:00:00 2001 From: Gustavo de Morais Date: Wed, 26 Nov 2025 15:28:34 +0000 Subject: [PATCH 19/31] [FLINK-38576][table] Align commonJoinKey in MultiJoin for logical and physical rules Co-authored-by: Stepan Stepanishchev <116617250+SteveStevenpoor@users.noreply.github.com> --- .../table/utils/NoCommonJoinKeyException.java | 40 + .../rules/logical/JoinToMultiJoinRule.java | 168 +- .../stream/StreamPhysicalMultiJoinRule.java | 126 +- .../planner/plan/utils/MultiJoinUtil.java | 155 ++ .../plan/stream/sql/MultiJoinTest.java | 1349 ++++++++-- .../planner/plan/stream/sql/MultiJoinTest.xml | 2256 ++++++++++++++--- .../AttributeBasedJoinKeyExtractor.java | 3 +- 7 files changed, 3284 insertions(+), 813 deletions(-) create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/NoCommonJoinKeyException.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/MultiJoinUtil.java diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/NoCommonJoinKeyException.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/NoCommonJoinKeyException.java new file mode 100644 index 0000000000000..21eb273b91171 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/NoCommonJoinKeyException.java @@ -0,0 +1,40 @@ +/* + * 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.utils; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.util.FlinkRuntimeException; + +/** Thrown when a MultiJoin node has no common join key. */ +@Internal +public class NoCommonJoinKeyException extends FlinkRuntimeException { + private static final long serialVersionUID = 1L; + + public NoCommonJoinKeyException(String message) { + super(message); + } + + public NoCommonJoinKeyException(String message, Throwable cause) { + super(message, cause); + } + + public NoCommonJoinKeyException(Throwable cause) { + super(cause); + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/JoinToMultiJoinRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/JoinToMultiJoinRule.java index b0e1c7de8e526..81f27d29788ab 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/JoinToMultiJoinRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/JoinToMultiJoinRule.java @@ -18,15 +18,18 @@ package org.apache.flink.table.planner.plan.rules.logical; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.table.api.TableException; +import org.apache.flink.table.planner.calcite.FlinkTypeFactory; import org.apache.flink.table.planner.hint.FlinkHints; import org.apache.flink.table.planner.hint.StateTtlHint; import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMultiJoin; import org.apache.flink.table.planner.plan.utils.IntervalJoinUtil; +import org.apache.flink.table.runtime.operators.join.stream.keyselector.AttributeBasedJoinKeyExtractor; +import org.apache.flink.table.runtime.operators.join.stream.keyselector.JoinKeyExtractor; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.utils.NoCommonJoinKeyException; import org.apache.calcite.plan.RelOptRuleCall; -import org.apache.calcite.plan.RelOptTable; import org.apache.calcite.plan.RelOptUtil; import org.apache.calcite.plan.RelRule; import org.apache.calcite.plan.hep.HepRelVertex; @@ -36,21 +39,15 @@ 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.core.TableFunctionScan; -import org.apache.calcite.rel.core.TableScan; -import org.apache.calcite.rel.core.Values; import org.apache.calcite.rel.hint.RelHint; import org.apache.calcite.rel.logical.LogicalJoin; import org.apache.calcite.rel.logical.LogicalSnapshot; -import org.apache.calcite.rel.metadata.RelColumnOrigin; -import org.apache.calcite.rel.metadata.RelMetadataQuery; import org.apache.calcite.rel.rules.CoreRules; import org.apache.calcite.rel.rules.FilterMultiJoinMergeRule; import org.apache.calcite.rel.rules.MultiJoin; import org.apache.calcite.rel.rules.ProjectMultiJoinMergeRule; import org.apache.calcite.rel.rules.TransformationRule; import org.apache.calcite.rex.RexBuilder; -import org.apache.calcite.rex.RexCall; import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexNode; import org.apache.calcite.rex.RexUtil; @@ -65,14 +62,14 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Set; import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.apache.flink.table.planner.hint.StateTtlHint.STATE_TTL; +import static org.apache.flink.table.planner.plan.utils.MultiJoinUtil.createJoinAttributeMap; /** * Flink Planner rule to flatten a tree of {@link Join}s into a single {@link MultiJoin} with N @@ -442,134 +439,45 @@ private boolean canCombine(RelNode input, Join origJoin) { /** * Checks if original join and child multi-join have common join keys to decide if we can merge - * them into a single MultiJoin with one more input. + * them into a single MultiJoin with one more input. The method uses {@link + * AttributeBasedJoinKeyExtractor} to try to create valid common join key extractors. * * @param origJoin original Join * @param otherJoin child MultiJoin * @return true if original Join and child multi-join have at least one common JoinKey */ private boolean haveCommonJoinKey(Join origJoin, MultiJoin otherJoin) { - Set origJoinKeys = getJoinKeys(origJoin); - Set otherJoinKeys = getJoinKeys(otherJoin); - - origJoinKeys.retainAll(otherJoinKeys); - - return !origJoinKeys.isEmpty(); - } - - /** - * Returns a set of join keys as strings following this format [table_name.field_name]. - * - * @param join Join or MultiJoin node - * @return set of all the join keys (keys from join conditions) - */ - public Set getJoinKeys(RelNode join) { - Set joinKeys = new HashSet<>(); - List conditions = Collections.emptyList(); - List inputs = join.getInputs(); - - if (join instanceof Join) { - conditions = collectConjunctions(((Join) join).getCondition()); - } else if (join instanceof MultiJoin) { - conditions = - ((MultiJoin) join) - .getOuterJoinConditions().stream() - .flatMap(cond -> collectConjunctions(cond).stream()) - .collect(Collectors.toList()); + final List combinedJoinInputs = + Stream.concat(otherJoin.getInputs().stream(), Stream.of(origJoin.getRight())) + .collect(Collectors.toUnmodifiableList()); + + final List combinedInputTypes = + combinedJoinInputs.stream() + .map(i -> FlinkTypeFactory.toLogicalRowType(i.getRowType())) + .collect(Collectors.toUnmodifiableList()); + + final List combinedJoinConditions = + Stream.concat( + otherJoin.getOuterJoinConditions().stream(), + List.of(origJoin.getCondition()).stream()) + .collect(Collectors.toUnmodifiableList()); + + final Map> + joinAttributeMap = + createJoinAttributeMap(combinedJoinInputs, combinedJoinConditions); + + boolean haveCommonJoinKey = false; + try { + // we probe to instantiate AttributeBasedJoinKeyExtractor's constructor to check whether + // it's possible to initialize common join key structures + final JoinKeyExtractor keyExtractor = + new AttributeBasedJoinKeyExtractor(joinAttributeMap, combinedInputTypes); + haveCommonJoinKey = keyExtractor.getCommonJoinKeyIndices(0).length > 0; + } catch (NoCommonJoinKeyException ignored) { + // failed to instantiate common join key structures => no common join key } - RelMetadataQuery mq = join.getCluster().getMetadataQuery(); - - for (RexCall condition : conditions) { - for (RexNode operand : condition.getOperands()) { - if (operand instanceof RexInputRef) { - addJoinKeysByOperand((RexInputRef) operand, inputs, mq, joinKeys); - } - } - } - - return joinKeys; - } - - /** - * Retrieves conjunctions from joinCondition. - * - * @param joinCondition join condition - * @return List of RexCalls representing conditions - */ - private List collectConjunctions(RexNode joinCondition) { - return RelOptUtil.conjunctions(joinCondition).stream() - .map(rexNode -> (RexCall) rexNode) - .collect(Collectors.toList()); - } - - /** - * Appends join key's string representation to the set of join keys. - * - * @param ref input ref to the operand - * @param inputs List of node's inputs - * @param mq RelMetadataQuery needed to retrieve column origins - * @param joinKeys Set of join keys to be added - */ - private void addJoinKeysByOperand( - RexInputRef ref, List inputs, RelMetadataQuery mq, Set joinKeys) { - int inputRefIndex = ref.getIndex(); - Tuple2 targetInputAndIdx = getTargetInputAndIdx(inputRefIndex, inputs); - RelNode targetInput = targetInputAndIdx.f0; - int idxInTargetInput = targetInputAndIdx.f1; - - Set origins = mq.getColumnOrigins(targetInput, idxInTargetInput); - if (origins != null) { - for (RelColumnOrigin origin : origins) { - RelOptTable originTable = origin.getOriginTable(); - List qualifiedName = originTable.getQualifiedName(); - String fieldName = - originTable - .getRowType() - .getFieldList() - .get(origin.getOriginColumnOrdinal()) - .getName(); - joinKeys.add(qualifiedName.get(qualifiedName.size() - 1) + "." + fieldName); - } - } - } - - /** - * Get real table that contains needed input ref (join key). - * - * @param inputRefIndex index of the required field - * @param inputs inputs of the node - * @return target input + idx of the required field as target input's - */ - private Tuple2 getTargetInputAndIdx(int inputRefIndex, List inputs) { - RelNode targetInput = null; - int idxInTargetInput = 0; - int inputFieldEnd = 0; - for (RelNode input : inputs) { - inputFieldEnd += input.getRowType().getFieldCount(); - if (inputRefIndex < inputFieldEnd) { - targetInput = input; - int targetInputStartIdx = inputFieldEnd - input.getRowType().getFieldCount(); - idxInTargetInput = inputRefIndex - targetInputStartIdx; - break; - } - } - - targetInput = - (targetInput instanceof HepRelVertex) - ? ((HepRelVertex) targetInput).getCurrentRel() - : targetInput; - - assert targetInput != null; - - if (targetInput instanceof TableScan - || targetInput instanceof Values - || targetInput instanceof TableFunctionScan - || targetInput.getInputs().isEmpty()) { - return new Tuple2<>(targetInput, idxInTargetInput); - } else { - return getTargetInputAndIdx(idxInTargetInput, targetInput.getInputs()); - } + return haveCommonJoinKey; } /** diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalMultiJoinRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalMultiJoinRule.java index 84b5de014b096..da3b99d669a73 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalMultiJoinRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalMultiJoinRule.java @@ -32,18 +32,14 @@ import org.apache.calcite.plan.RelTraitSet; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.convert.ConverterRule; -import org.apache.calcite.rex.RexCall; -import org.apache.calcite.rex.RexInputRef; -import org.apache.calcite.rex.RexNode; -import org.apache.calcite.sql.SqlKind; -import org.checkerframework.checker.nullness.qual.Nullable; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import static org.apache.flink.table.planner.plan.utils.MultiJoinUtil.createJoinAttributeMap; + /** Rule that converts {@link FlinkLogicalMultiJoin} to {@link StreamPhysicalMultiJoin}. */ public class StreamPhysicalMultiJoinRule extends ConverterRule { public static final RelOptRule INSTANCE = new StreamPhysicalMultiJoinRule(); @@ -61,7 +57,7 @@ private StreamPhysicalMultiJoinRule() { public RelNode convert(final RelNode rel) { final FlinkLogicalMultiJoin multiJoin = (FlinkLogicalMultiJoin) rel; final Map> joinAttributeMap = - createJoinAttributeMap(multiJoin); + createJoinAttributeMap(multiJoin.getInputs(), multiJoin.getJoinConditions()); final List inputRowTypes = multiJoin.getInputs().stream() .map(i -> FlinkTypeFactory.toLogicalRowType(i.getRowType())) @@ -117,120 +113,4 @@ private RelTraitSet createInputTraitSet( return inputTraitSet; } - - private Map> createJoinAttributeMap( - final FlinkLogicalMultiJoin multiJoin) { - final Map> joinAttributeMap = new HashMap<>(); - final List inputFieldCounts = - multiJoin.getInputs().stream() - .map(input -> input.getRowType().getFieldCount()) - .collect(Collectors.toList()); - - final List inputOffsets = new ArrayList<>(); - int currentOffset = 0; - for (final Integer count : inputFieldCounts) { - inputOffsets.add(currentOffset); - currentOffset += count; - } - - final List joinConditions = multiJoin.getJoinConditions(); - for (final RexNode condition : joinConditions) { - extractEqualityConditions(condition, inputOffsets, inputFieldCounts, joinAttributeMap); - } - return joinAttributeMap; - } - - private void extractEqualityConditions( - final RexNode condition, - final List inputOffsets, - final List inputFieldCounts, - final Map> joinAttributeMap) { - if (!(condition instanceof RexCall)) { - return; - } - - final RexCall call = (RexCall) condition; - final SqlKind kind = call.getOperator().getKind(); - - if (kind != SqlKind.EQUALS) { - for (final RexNode operand : call.getOperands()) { - extractEqualityConditions( - operand, inputOffsets, inputFieldCounts, joinAttributeMap); - } - return; - } - - if (call.getOperands().size() != 2) { - return; - } - - final RexNode op1 = call.getOperands().get(0); - final RexNode op2 = call.getOperands().get(1); - - if (!(op1 instanceof RexInputRef) || !(op2 instanceof RexInputRef)) { - return; - } - - final InputRef inputRef1 = - findInputRef(((RexInputRef) op1).getIndex(), inputOffsets, inputFieldCounts); - final InputRef inputRef2 = - findInputRef(((RexInputRef) op2).getIndex(), inputOffsets, inputFieldCounts); - - if (inputRef1 == null || inputRef2 == null) { - return; - } - - final InputRef leftRef; - final InputRef rightRef; - if (inputRef1.inputIndex < inputRef2.inputIndex) { - leftRef = inputRef1; - rightRef = inputRef2; - } else { - leftRef = inputRef2; - rightRef = inputRef1; - } - - // Special case for input 0: - // Since we are building attribute references that do left -> right index, - // we need a special base case for input 0 which has no input to the left. - // So we do {-1, -1} -> {0, attributeIndex} - if (leftRef.inputIndex == 0) { - final ConditionAttributeRef firstAttrRef = - new ConditionAttributeRef(-1, -1, leftRef.inputIndex, leftRef.attributeIndex); - joinAttributeMap - .computeIfAbsent(leftRef.inputIndex, k -> new ArrayList<>()) - .add(firstAttrRef); - } - - final ConditionAttributeRef attrRef = - new ConditionAttributeRef( - leftRef.inputIndex, - leftRef.attributeIndex, - rightRef.inputIndex, - rightRef.attributeIndex); - joinAttributeMap.computeIfAbsent(rightRef.inputIndex, k -> new ArrayList<>()).add(attrRef); - } - - private @Nullable InputRef findInputRef( - final int fieldIndex, - final List inputOffsets, - final List inputFieldCounts) { - for (int i = 0; i < inputOffsets.size(); i++) { - final int offset = inputOffsets.get(i); - if (fieldIndex >= offset && fieldIndex < offset + inputFieldCounts.get(i)) { - return new InputRef(i, fieldIndex - offset); - } - } - return null; - } - - private static final class InputRef { - private final int inputIndex; - private final int attributeIndex; - - private InputRef(final int inputIndex, final int attributeIndex) { - this.inputIndex = inputIndex; - this.attributeIndex = attributeIndex; - } - } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/MultiJoinUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/MultiJoinUtil.java new file mode 100644 index 0000000000000..83bf9be50c1a3 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/MultiJoinUtil.java @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.utils; + +import org.apache.flink.table.runtime.operators.join.stream.keyselector.AttributeBasedJoinKeyExtractor; + +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexInputRef; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.sql.SqlKind; +import org.checkerframework.checker.nullness.qual.Nullable; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class MultiJoinUtil { + public static Map> + createJoinAttributeMap( + List joinInputs, List joinConditions) { + final Map> + joinAttributeMap = new HashMap<>(); + final List inputFieldCounts = + joinInputs.stream() + .map(input -> input.getRowType().getFieldCount()) + .collect(Collectors.toList()); + + final List inputOffsets = new ArrayList<>(); + int currentOffset = 0; + for (final Integer count : inputFieldCounts) { + inputOffsets.add(currentOffset); + currentOffset += count; + } + + for (final RexNode condition : joinConditions) { + extractEqualityConditions(condition, inputOffsets, inputFieldCounts, joinAttributeMap); + } + return joinAttributeMap; + } + + private static void extractEqualityConditions( + final RexNode condition, + final List inputOffsets, + final List inputFieldCounts, + final Map> + joinAttributeMap) { + if (!(condition instanceof RexCall)) { + return; + } + + final RexCall call = (RexCall) condition; + final SqlKind kind = call.getOperator().getKind(); + + if (kind != SqlKind.EQUALS) { + for (final RexNode operand : call.getOperands()) { + extractEqualityConditions( + operand, inputOffsets, inputFieldCounts, joinAttributeMap); + } + return; + } + + if (call.getOperands().size() != 2) { + return; + } + + final RexNode op1 = call.getOperands().get(0); + final RexNode op2 = call.getOperands().get(1); + + if (!(op1 instanceof RexInputRef) || !(op2 instanceof RexInputRef)) { + return; + } + + final InputRef inputRef1 = + findInputRef(((RexInputRef) op1).getIndex(), inputOffsets, inputFieldCounts); + final InputRef inputRef2 = + findInputRef(((RexInputRef) op2).getIndex(), inputOffsets, inputFieldCounts); + + if (inputRef1 == null || inputRef2 == null) { + return; + } + + final InputRef leftRef; + final InputRef rightRef; + if (inputRef1.inputIndex < inputRef2.inputIndex) { + leftRef = inputRef1; + rightRef = inputRef2; + } else { + leftRef = inputRef2; + rightRef = inputRef1; + } + + // Special case for input 0: + // Since we are building attribute references that do left -> right index, + // we need a special base case for input 0 which has no input to the left. + // So we do {-1, -1} -> {0, attributeIndex} + if (leftRef.inputIndex == 0) { + final AttributeBasedJoinKeyExtractor.ConditionAttributeRef firstAttrRef = + new AttributeBasedJoinKeyExtractor.ConditionAttributeRef( + -1, -1, leftRef.inputIndex, leftRef.attributeIndex); + joinAttributeMap + .computeIfAbsent(leftRef.inputIndex, k -> new ArrayList<>()) + .add(firstAttrRef); + } + + final AttributeBasedJoinKeyExtractor.ConditionAttributeRef attrRef = + new AttributeBasedJoinKeyExtractor.ConditionAttributeRef( + leftRef.inputIndex, + leftRef.attributeIndex, + rightRef.inputIndex, + rightRef.attributeIndex); + joinAttributeMap.computeIfAbsent(rightRef.inputIndex, k -> new ArrayList<>()).add(attrRef); + } + + private static @Nullable InputRef findInputRef( + final int fieldIndex, + final List inputOffsets, + final List inputFieldCounts) { + for (int i = 0; i < inputOffsets.size(); i++) { + final int offset = inputOffsets.get(i); + if (fieldIndex >= offset && fieldIndex < offset + inputFieldCounts.get(i)) { + return new InputRef(i, fieldIndex - offset); + } + } + return null; + } + + private static final class InputRef { + private final int inputIndex; + private final int attributeIndex; + + private InputRef(final int inputIndex, final int attributeIndex) { + this.inputIndex = inputIndex; + this.attributeIndex = attributeIndex; + } + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.java index a832696fbda74..81cc1a10ef9e3 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.java @@ -27,6 +27,7 @@ import org.apache.flink.table.planner.utils.TableTestBase; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import scala.Enumeration; @@ -50,7 +51,7 @@ void setup() { util.tableEnv() .executeSql( "CREATE TABLE Users (" - + " user_id_0 STRING PRIMARY KEY NOT ENFORCED," + + " user_id STRING PRIMARY KEY NOT ENFORCED," + " name STRING," + " cash INT" + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); @@ -59,7 +60,7 @@ void setup() { .executeSql( "CREATE TABLE Orders (" + " order_id STRING PRIMARY KEY NOT ENFORCED," - + " user_id_1 STRING," + + " user_id STRING," + " product STRING" + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,D')"); @@ -68,16 +69,26 @@ void setup() { "CREATE TABLE Payments (" + " payment_id STRING PRIMARY KEY NOT ENFORCED," + " price INT," - + " user_id_2 STRING" + + " user_id STRING" + ") WITH ('connector' = 'values', 'changelog-mode' = 'I')"); util.tableEnv() .executeSql( "CREATE TABLE Shipments (" + " location STRING," - + " user_id_3 STRING" + + " user_id STRING" + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,UB,D')"); + util.tableEnv() + .executeSql( + "CREATE TABLE Detail (" + + " detail_id STRING PRIMARY KEY NOT ENFORCED," + + " description STRING," + + " user_id STRING," + + " data STRING," + + " `timestamp` BIGINT" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I')"); + // Tables for testing temporal join exclusion util.tableEnv() .executeSql( @@ -101,8 +112,8 @@ void setup() { "CREATE TABLE EventTable1 (" + " id STRING," + " val INT," - + " rowtime TIMESTAMP(3)," - + " WATERMARK FOR rowtime AS rowtime - INTERVAL '5' SECOND" + + " `$rowtime` TIMESTAMP(3)," + + " WATERMARK FOR `$rowtime` AS `$rowtime` - INTERVAL '5' SECOND" + ") WITH ('connector' = 'values', 'changelog-mode' = 'I')"); util.tableEnv() @@ -110,15 +121,15 @@ void setup() { "CREATE TABLE EventTable2 (" + " id STRING," + " price DOUBLE," - + " rowtime TIMESTAMP(3)," - + " WATERMARK FOR rowtime AS rowtime - INTERVAL '5' SECOND" + + " `$rowtime` TIMESTAMP(3)," + + " WATERMARK FOR `$rowtime` AS `$rowtime` - INTERVAL '5' SECOND" + ") WITH ('connector' = 'values', 'changelog-mode' = 'I')"); // Tables for testing time attribute materialization in multi-join util.tableEnv() .executeSql( "CREATE TABLE UsersWithProctime (" - + " user_id_0 STRING PRIMARY KEY NOT ENFORCED," + + " user_id STRING PRIMARY KEY NOT ENFORCED," + " name STRING," + " proctime AS PROCTIME()" + ") WITH ('connector' = 'values', 'changelog-mode' = 'I')"); @@ -127,9 +138,9 @@ void setup() { .executeSql( "CREATE TABLE OrdersWithRowtime (" + " order_id STRING PRIMARY KEY NOT ENFORCED," - + " user_id_1 STRING," - + " rowtime TIMESTAMP(3)," - + " WATERMARK FOR rowtime AS rowtime" + + " user_id STRING," + + " `$rowtime` TIMESTAMP(3)," + + " WATERMARK FOR `$rowtime` AS `$rowtime`" + ") WITH ('connector' = 'values', 'changelog-mode' = 'I')"); // Tables for testing upsert key preservation util.tableEnv() @@ -170,64 +181,109 @@ void setup() { @Test void testThreeWayInnerJoinRelPlan() { util.verifyRelPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "INNER JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "INNER JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "INNER JOIN Payments p\n" + + " ON u.user_id = p.user_id"); } @Test - void testThreeWayInnerJoinNoCommonJoinKeyRelPlan() { + @Tag("no-common-join-key") + void testThreeWayInnerJoinRelPlanNoCommonJoinKey() { util.verifyRelPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "INNER JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "INNER JOIN Payments p ON u.cash = p.price"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "INNER JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "INNER JOIN Payments p\n" + + " ON u.cash = p.price"); } @Test void testThreeWayInnerJoinExecPlan() { util.verifyExecPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "INNER JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "INNER JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "INNER JOIN Payments p\n" + + " ON u.user_id = p.user_id"); } @Test void testThreeWayLeftOuterJoinRelPlan() { util.verifyRelPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "LEFT JOIN Payments p ON u.user_id_0 = p.user_id_2"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id"); } @Test void testThreeWayInnerJoinWithTttlHints() { util.verifyRelPlan( - "SELECT /*+ STATE_TTL(u='1d', o='2d', p='1h') */u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "INNER JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2"); + "\nSELECT\n" + + " /*+ STATE_TTL(u='1d', o='2d', p='1h') */\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "INNER JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "INNER JOIN Payments p\n" + + " ON u.user_id = p.user_id"); } @Test void testThreeWayInnerJoinWithSingleTttlHint() { util.verifyRelPlan( - "SELECT /*+ STaTE_tTL(o='2d') */u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "INNER JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2"); + "\nSELECT\n" + + " /*+ STaTE_tTL(o='2d') */\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "INNER JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "INNER JOIN Payments p\n" + + " ON u.user_id = p.user_id"); } @Test void testThreeWayLeftOuterJoinExecPlan() { util.verifyExecPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "LEFT JOIN Payments p ON u.user_id_0 = p.user_id_2"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id"); } @Test @@ -236,154 +292,241 @@ void testTwoWayJoinWithUnion() { .executeSql( "CREATE TABLE Orders2 (" + " order_id STRING PRIMARY KEY NOT ENFORCED," - + " user_id_1 STRING," + + " user_id STRING," + " product STRING" + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,D')"); util.verifyRelPlan( - "WITH OrdersUnion as (" - + "SELECT * FROM Orders " - + "UNION ALL " - + "SELECT * FROM Orders2" - + ") " - + "SELECT * FROM OrdersUnion o " - + "LEFT JOIN Users u " - + "ON o.user_id_1 = u.user_id_0"); + "\nWITH OrdersUnion as (\n" + + "SELECT * FROM Orders\n" + + "UNION ALL\n" + + "SELECT * FROM Orders2\n" + + ")\n" + + "SELECT * FROM OrdersUnion o\n" + + "LEFT JOIN Users u\n" + + " ON o.user_id = u.user_id"); } @Test void testTwoWayJoinWithRank() { - util.getTableEnv() - .getConfig() - .set(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true); - - util.verifyRelPlan( - "WITH JoinedEvents as (" - + "SELECT e1.id as id, e1.val, e1.rowtime as `rowtime`, e2.price " - + "FROM EventTable1 e1 " - + "JOIN EventTable2 e2 ON e1.id = e2.id) " - + "SELECT id, val, `rowtime` FROM (" - + "SELECT *, " - + "ROW_NUMBER() OVER (PARTITION BY id ORDER BY `rowtime` DESC) as ts " - + "FROM JoinedEvents) " + util.verifyRelPlan( + "\nWITH JoinedEvents as (\n" + + "SELECT\n" + + " e1.id as id,\n" + + " e1.val,\n" + + " e1.`$rowtime` as `$rowtime`,\n" + + " e2.price\n" + + "FROM EventTable1 e1\n" + + "JOIN EventTable2 e2\n" + + " ON e1.id = e2.id)\n" + + "SELECT\n" + + " id,\n" + + " val,\n" + + " `$rowtime`\n" + + "FROM (\n" + + " SELECT\n" + + " *,\n" + + " ROW_NUMBER() OVER (PARTITION BY id ORDER BY `$rowtime` DESC) as ts\n" + + " FROM JoinedEvents)\n" + "WHERE ts = 1"); } @Test void testFourWayComplexJoinRelPlan() { util.verifyRelPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id, s.location " - + "FROM Users u " - + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2 AND (u.cash >= p.price OR p.price < 0) " - + "LEFT JOIN Shipments s ON p.user_id_2 = s.user_id_3"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id,\n" + + " s.location\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "INNER JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + + " AND (u.cash >= p.price OR p.price < 0)\n" + + "LEFT JOIN Shipments s\n" + + " ON p.user_id = s.user_id"); } @Test - void testThreeWayJoinNoJoinKeyExecPlan() { + @Tag("no-common-join-key") + void testThreeWayJoinExecPlanNoCommonJoinKey() { util.verifyExecPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "LEFT JOIN Orders o ON TRUE " - + "INNER JOIN Payments p ON TRUE "); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o ON TRUE\n" + + "INNER JOIN Payments p ON TRUE"); } @Test - void testFourWayJoinNoCommonJoinKeyRelPlan() { + @Tag("no-common-join-key") + void testFourWayJoinRelPlanNoCommonJoinKey() { util.verifyRelPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id, s.location " - + "FROM Users u " - + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2 " - + "LEFT JOIN Shipments s ON p.payment_id = s.user_id_3"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id,\n" + + " s.location\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN LookupTable\n" + + " ON u.name = LookupTable.name\n" + + "LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + + "LEFT JOIN Shipments s\n" + + " ON o.user_id = s.user_id"); } @Test void testFourWayComplexJoinExecPlan() { util.verifyExecPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id, s.location " - + "FROM Users u " - + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2 AND (u.cash >= p.price OR p.price < 0) " - + "LEFT JOIN Shipments s ON p.user_id_2 = s.user_id_3"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id,\n" + + " s.location\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "INNER JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + + " AND (u.cash >= p.price OR p.price < 0)\n" + + "LEFT JOIN Shipments s\n" + + " ON p.user_id = s.user_id"); } @Test void testThreeWayInnerJoinExplain() { util.verifyExplain( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "INNER JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "INNER JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "INNER JOIN Payments p\n" + + " ON u.user_id = p.user_id"); } @Test void testThreeWayLeftOuterJoinExplain() { util.verifyExplain( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "LEFT JOIN Payments p ON u.user_id_0 = p.user_id_2"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id"); } @Test void testFourWayComplexJoinExplain() { util.verifyExplain( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id, s.location " - + "FROM Users u " - + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2 AND (u.cash >= p.price OR p.price < 0) " - + "LEFT JOIN Shipments s ON p.user_id_2 = s.user_id_3"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id,\n" + + " s.location\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "INNER JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + + " AND (u.cash >= p.price OR p.price < 0)\n" + + "LEFT JOIN Shipments s\n" + + " ON p.user_id = s.user_id"); } @Test void testTemporalJoinExcludedFromMultiJoin() { // Temporal joins should remain as lookup joins, not be merged into MultiJoin util.verifyRelPlan( - "SELECT s.user_id, s.amount, l.name, l.age " - + "FROM StreamTable s " - + "JOIN LookupTable FOR SYSTEM_TIME AS OF s.proctime AS l " - + "ON s.user_id = l.id"); + "\nSELECT\n" + + " s.user_id,\n" + + " s.amount,\n" + + " l.name,\n" + + " l.age\n" + + "FROM StreamTable s\n" + + "JOIN LookupTable FOR SYSTEM_TIME AS OF s.proctime AS l\n" + + " ON s.user_id = l.id"); } @Test void testIntervalJoinExcludedFromMultiJoin() { // Interval joins (event-time and processing-time) should remain as interval joins util.verifyRelPlan( - "SELECT e1.id, e1.val, e2.price " - + "FROM EventTable1 e1 " - + "JOIN EventTable2 e2 ON e1.id = e2.id " - + "AND e1.rowtime BETWEEN e2.rowtime - INTERVAL '1' MINUTE " - + "AND e2.rowtime + INTERVAL '1' MINUTE"); + "\nSELECT\n" + + " e1.id,\n" + + " e1.val,\n" + + " e2.price\n" + + "FROM EventTable1 e1\n" + + "JOIN EventTable2 e2\n" + + " ON e1.id = e2.id\n" + + " AND e1.`$rowtime` BETWEEN e2.`$rowtime` - INTERVAL '1' MINUTE\n" + + " AND e2.`$rowtime` + INTERVAL '1' MINUTE"); } @Test void testThreeWayLeftOuterJoinWithWhereClauseRelPlan() { util.verifyRelPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "LEFT JOIN Payments p ON u.user_id_0 = p.user_id_2 " + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + "WHERE u.name = 'Gus' AND p.price > 10"); } @Test void testThreeWayLeftOuterJoinWithWhereClauseExecPlan() { util.verifyExecPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "LEFT JOIN Payments p ON u.user_id_0 = p.user_id_2 " + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + "WHERE u.name = 'Gus' AND p.price > 10"); } @Test void testThreeWayLeftOuterJoinWithWhereClauseExplain() { util.verifyExplain( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "LEFT JOIN Payments p ON u.user_id_0 = p.user_id_2 " + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + "WHERE u.name = 'Gus' AND p.price > 10"); } @@ -391,53 +534,84 @@ void testThreeWayLeftOuterJoinWithWhereClauseExplain() { void testRegularJoinsAreMergedApartFromTemporalJoin() { // Regular joins should still be eligible for MultiJoin but not mixed with temporal joins util.verifyRelPlan( - "SELECT u.user_id_0, u.name, o.order_id, temporal.age " - + "FROM Users u " - + "INNER JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "INNER JOIN (" - + " SELECT s.user_id, l.age " - + " FROM StreamTable s " - + " JOIN LookupTable FOR SYSTEM_TIME AS OF s.proctime AS l " - + " ON s.user_id = l.id" - + ") temporal ON u.user_id_0 = temporal.user_id"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " temporal.age " + + "FROM Users u\n" + + "INNER JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "INNER JOIN (\n" + + " SELECT s.user_id, l.age\n" + + " FROM StreamTable s\n" + + " JOIN LookupTable FOR SYSTEM_TIME AS OF s.proctime AS l\n" + + " ON s.user_id = l.id\n" + + ") temporal ON u.user_id = temporal.user_id"); } @Test void testFourWayJoinTransitiveCommonJoinKeyRelPlan() { util.verifyRelPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id, s.location " - + "FROM Users u " - + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "LEFT JOIN Payments p ON o.user_id_1 = p.user_id_2 " - + "LEFT JOIN Shipments s ON p.user_id_2 = s.user_id_3"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id,\n" + + " s.location\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON o.user_id = p.user_id\n" + + "LEFT JOIN Shipments s\n" + + " ON p.user_id = s.user_id"); } /* Update this to supported with FLINK-37973 https://issues.apache.org/jira/browse/FLINK-37973 */ @Test void testRightJoinNotSupported() { util.verifyRelPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "RIGHT JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "RIGHT JOIN Payments p ON o.user_id_1 = p.user_id_2"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "RIGHT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "RIGHT JOIN Payments p\n" + + " ON o.user_id = p.user_id"); } @Test void testFullOuterNotSupported() { util.verifyRelPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "FULL OUTER JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "FULL OUTER JOIN Payments p ON o.user_id_1 = p.user_id_2"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "FULL OUTER JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "FULL OUTER JOIN Payments p\n" + + " ON o.user_id = p.user_id"); } @Test void testThreeWayJoinWithTimeAttributesMaterialization() { util.verifyRelPlan( - "SELECT u.name, u.proctime, o.rowtime, p.price " - + "FROM UsersWithProctime u " - + "JOIN OrdersWithRowtime o ON u.user_id_0 = o.user_id_1 " - + "JOIN Payments p ON u.user_id_0 = p.user_id_2"); + "\nSELECT\n" + + " u.name,\n" + + " u.proctime,\n" + + " o.`$rowtime`,\n" + + " p.price\n" + + "FROM UsersWithProctime u\n" + + "JOIN OrdersWithRowtime o\n" + + " ON u.user_id = o.user_id\n" + + "JOIN Payments p\n" + + " ON u.user_id = p.user_id"); } @Test @@ -456,15 +630,15 @@ void testPreservesUpsertKeyTwoWayLeftJoinOrders() { + ")"); util.verifyRelPlanInsert( - "INSERT INTO sink_two_way " - + "SELECT" - + " o.user_id," - + " o.order_id," - + " o.product," - + " u.region_id " - + "FROM OrdersPK o " - + "LEFT JOIN UsersPK u" - + " ON u.user_id = o.user_id"); + "\nINSERT INTO sink_two_way\n" + + "SELECT\n" + + " o.user_id,\n" + + " o.order_id,\n" + + " o.product,\n" + + " u.region_id\n" + + "FROM OrdersPK o\n" + + "LEFT JOIN UsersPK u\n" + + " ON u.user_id = o.user_id"); } @Test @@ -483,15 +657,15 @@ void testPreservesUpsertKeyTwoWayInnerJoinOrders() { + ")"); util.verifyRelPlanInsert( - "INSERT INTO sink_two_way " - + "SELECT" - + " o.user_id," - + " o.order_id," - + " o.product," - + " u.region_id " - + "FROM UsersPK u " - + "INNER JOIN OrdersPK o " - + " ON u.user_id = o.user_id"); + "\nINSERT INTO sink_two_way\n" + + "SELECT\n" + + " o.user_id,\n" + + " o.order_id,\n" + + " o.product,\n" + + " u.region_id\n" + + "FROM UsersPK u\n" + + "INNER JOIN OrdersPK o\n" + + " ON u.user_id = o.user_id"); } @Test @@ -519,15 +693,15 @@ void testPreservesUpsertKeyTwoWayInnerJoinOrdersDoesNot() { + ")"); util.verifyRelPlanInsert( - "INSERT INTO sink_two_way " - + "SELECT" - + " o.user_id," - + " o.order_id," - + " o.product," - + " u.region_id " - + "FROM UsersPK u " - + "INNER JOIN OrdersSimplePK o " - + " ON u.user_id = o.user_id"); + "\nINSERT INTO sink_two_way\n" + + "SELECT\n" + + " o.user_id,\n" + + " o.order_id,\n" + + " o.product,\n" + + " u.region_id\n" + + "FROM UsersPK u\n" + + "INNER JOIN OrdersSimplePK o\n" + + " ON u.user_id = o.user_id"); } @Test @@ -548,19 +722,19 @@ void testPreservesUpsertKeyThreeWayJoin() { + ")"); util.verifyRelPlanInsert( - "INSERT INTO sink_three_way " - + "SELECT" - + " o.user_id," - + " o.order_id," - + " p.user_id," - + " p.payment_id," - + " u.user_id," - + " u.description " - + "FROM UsersPK u " - + "JOIN OrdersPK o" - + " ON o.user_id = u.user_id " - + "JOIN PaymentsPK p" - + " ON o.user_id = p.user_id"); + "\nINSERT INTO sink_three_way\n" + + "SELECT\n" + + " o.user_id,\n" + + " o.order_id,\n" + + " p.user_id,\n" + + " p.payment_id,\n" + + " u.user_id,\n" + + " u.description\n" + + "FROM UsersPK u\n" + + "JOIN OrdersPK o\n" + + " ON o.user_id = u.user_id\n" + + "JOIN PaymentsPK p\n" + + " ON o.user_id = p.user_id"); } @Test @@ -568,36 +742,39 @@ void testPreservesUpsertKeyFourWayComplex() { util.tableEnv() .executeSql( "CREATE TABLE sink_four_way (" - + " user_id_0 STRING NOT NULL," + + " user_id STRING NOT NULL," + " order_id STRING NOT NULL," - + " user_id_1 STRING NOT NULL," + + " user_id1 STRING NOT NULL," + " payment_id STRING NOT NULL," - + " user_id_2 STRING NOT NULL," + + " user_id2 STRING NOT NULL," + " name STRING," + " location STRING," - + " CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id_0`, `order_id`, `user_id_1`, `payment_id`, `user_id_2`) NOT ENFORCED" + + " CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id`, `order_id`, `user_id1`, `payment_id`, `user_id2`) NOT ENFORCED" + ") WITH (" + " 'connector' = 'values'," + " 'sink-insert-only' = 'false'" + ")"); util.verifyRelPlanInsert( - "INSERT INTO sink_four_way " - + "SELECT" - + " u.user_id," - + " o.order_id," - + " o.user_id," - + " p.payment_id," - + " p.user_id," - + " u.name," - + " a.location " - + "FROM UsersPK u " - + "JOIN OrdersPK o" - + " ON u.user_id = o.user_id AND o.product IS NOT NULL " - + "JOIN PaymentsPK p" - + " ON u.user_id = p.user_id AND p.price >= 0 " - + "JOIN AddressPK a" - + " ON u.user_id = a.user_id AND a.location IS NOT NULL"); + "\nINSERT INTO sink_four_way\n" + + "SELECT\n" + + " u.user_id,\n" + + " o.order_id,\n" + + " o.user_id,\n" + + " p.payment_id,\n" + + " p.user_id,\n" + + " u.name,\n" + + " a.location\n" + + "FROM UsersPK u\n" + + "JOIN OrdersPK o\n" + + " ON u.user_id = o.user_id\n" + + " AND o.product IS NOT NULL\n" + + "JOIN PaymentsPK p\n" + + " ON u.user_id = p.user_id\n" + + " AND p.price >= 0\n" + + "JOIN AddressPK a\n" + + " ON u.user_id = a.user_id\n" + + " AND a.location IS NOT NULL"); } @Test @@ -684,4 +861,768 @@ void testMultiSinkOnMultiJoinedView() { false, false); } + + /* + * Calcite adds a LogicalProject to compute expressions such as UPPER and FLOOR + * on the necessary fields. As a result, the planner cannot fuse all joins into + * a single MultiJoin node initially. + */ + @Test + @Tag("multijoin-chain-expected") + void testFourWayJoinWithFunctionInConditionMultiJoinChainExpected() { + util.verifyRelPlan( + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id,\n" + + " s.location\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON o.user_id = u.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + + " AND UPPER(u.name) = UPPER(p.payment_id)\n" + + " AND (FLOOR(u.cash) >= FLOOR(p.price) OR p.price < 0)\n" + + "LEFT JOIN Shipments s\n" + + " ON p.payment_id = s.location"); + } + + /* + * We expect the join inputs to **not** merge into a single MultiJoin node in this case, + * because `documents.common_id` is different from `other_documents.common_id`. + */ + @Test + @Tag("no-common-join-key") + void testComplexCommonJoinKeyMissingProjectionNoCommonJoinKey() { + util.tableEnv() + .executeSql( + "CREATE TABLE Assignments (" + + " assignment_id STRING PRIMARY KEY NOT ENFORCED," + + " user_id STRING," + + " detail_id STRING," + + " common_id STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE Documents (" + + " detail_id STRING PRIMARY KEY NOT ENFORCED," + + " creator_nm STRING," + + " common_id STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.verifyRelPlan( + "\nSELECT *\n" + + "FROM Assignments assignments\n" + + "LEFT JOIN Documents AS documents\n" + + " ON assignments.detail_id = documents.detail_id\n" + + " AND assignments.common_id = documents.common_id\n" + + "LEFT JOIN Documents AS other_documents\n" + + " ON assignments.user_id = other_documents.common_id"); + } + + @Test + void testComplexCommonJoinKey() { + util.tableEnv() + .executeSql( + "CREATE TABLE Assignments (" + + " assignment_id STRING PRIMARY KEY NOT ENFORCED," + + " user_id STRING," + + " detail_id STRING," + + " common_id STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE Customers (" + + " user_id STRING PRIMARY KEY NOT ENFORCED," + + " name STRING," + + " depart_num STRING," + + " common_id STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE Documents (" + + " detail_id STRING PRIMARY KEY NOT ENFORCED," + + " creator_nm STRING," + + " common_id STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE PhaseDetails (" + + " phase_id STRING PRIMARY KEY NOT ENFORCED," + + " common_id STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE Organizations (" + + " org_id STRING PRIMARY KEY NOT ENFORCED," + + " org_name STRING," + + " common_id STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.verifyExecPlan( + "\nSELECT *\n" + + "FROM Assignments assignments\n" + + "LEFT JOIN Customers AS customer\n" + + " ON assignments.user_id = customer.user_id\n" + + " AND assignments.common_id = customer.common_id\n" + + "LEFT JOIN Documents AS documents\n" + + " ON assignments.detail_id = documents.detail_id\n" + + " AND assignments.common_id = documents.common_id\n" + + "LEFT JOIN PhaseDetails AS phase_details\n" + + " ON documents.common_id = phase_details.common_id\n" + + "LEFT JOIN Organizations AS organizations\n" + + " ON customer.depart_num = organizations.org_id\n" + + " AND customer.common_id = organizations.common_id\n" + + "LEFT JOIN Customers AS creators\n" + + " ON documents.creator_nm = creators.depart_num\n" + + " AND documents.common_id = creators.common_id"); + } + + @Test + @Tag("no-common-join-key") + void testComplexConditionalLogicWithMultiJoinNoCommonJoinKey() { + util.tableEnv() + .executeSql( + "CREATE TABLE ProductCategories (" + + " category_id STRING PRIMARY KEY NOT ENFORCED," + + " category_name STRING," + + " is_premium BOOLEAN," + + " discount_rate DOUBLE" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE ProductReviews (" + + " review_id STRING PRIMARY KEY NOT ENFORCED," + + " product_id STRING," + + " rating INT," + + " is_verified BOOLEAN" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.verifyRelPlan( + "\nSELECT\n" + + " u.user_id,\n" + + " o.order_id,\n" + + " p.payment_id,\n" + + " pc.category_name,\n" + + " CASE\n" + + " WHEN pc.is_premium = true AND p.price > 1000 THEN 'High-Value Premium'\n" + + " WHEN pc.is_premium = true THEN 'Premium'\n" + + " WHEN p.price > 500 THEN 'Standard High-Value'\n" + + " ELSE 'Standard'\n" + + " END AS product_tier,\n" + + " CASE\n" + + " WHEN pr.rating >= 4 AND pr.is_verified = true THEN 'Highly Recommended'\n" + + " WHEN pr.rating >= 3 THEN 'Recommended'\n" + + " WHEN pr.rating >= 2 THEN 'Average'\n" + + " ELSE 'Not Recommended'\n" + + " END AS recommendation_status,\n" + + " CASE\n" + + " WHEN pc.discount_rate > 0.2 THEN p.price * (1 - pc.discount_rate)\n" + + " ELSE p.price\n" + + " END AS final_price\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + + "LEFT JOIN ProductCategories pc\n" + + " ON o.product = pc.category_id\n" + + "LEFT JOIN ProductReviews pr\n" + + " ON o.product = pr.product_id"); + } + + @Test + @Tag("no-common-join-key") + void testComplexCTEWithMultiJoinNoCommonJoinKey() { + util.tableEnv() + .executeSql( + "CREATE TABLE OrderStatus (" + + " status_id STRING PRIMARY KEY NOT ENFORCED," + + " status_name STRING," + + " is_final BOOLEAN" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE PaymentMethods (" + + " method_id STRING PRIMARY KEY NOT ENFORCED," + + " method_name STRING," + + " processing_fee DOUBLE" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.verifyRelPlan( + "\nWITH user_orders AS (\n" + + " SELECT u.user_id, u.name, o.order_id, o.product, p.payment_id, p.price\n" + + " FROM Users u\n" + + " LEFT JOIN Orders o ON\n" + + " u.user_id = o.user_id\n" + + " LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + + "),\n" + + "order_details AS (\n" + + " SELECT uo.*, os.status_name, os.is_final, pm.method_name, pm.processing_fee\n" + + " FROM user_orders uo\n" + + " LEFT JOIN OrderStatus os\n" + + " ON uo.order_id = os.status_id\n" + + " LEFT JOIN PaymentMethods pm\n" + + " ON uo.payment_id = pm.method_id\n" + + "),\n" + + "final_summary AS (\n" + + " SELECT\n" + + " user_id,\n" + + " name,\n" + + " COUNT(order_id) as total_orders,\n" + + " SUM(price) as total_spent,\n" + + " AVG(price) as avg_order_value,\n" + + " COUNT(CASE WHEN is_final = true THEN 1 END) as completed_orders\n" + + " FROM order_details\n" + + " GROUP BY user_id, name\n" + + ")\n" + + "SELECT * FROM final_summary"); + } + + @Test + @Tag("no-common-join-key") + void testAggregationAndGroupingWithMultiJoinNoCommonJoinKey() { + util.tableEnv() + .executeSql( + "CREATE TABLE OrderItems (" + + " item_id STRING PRIMARY KEY NOT ENFORCED," + + " order_id STRING," + + " product_name STRING," + + " quantity INT," + + " unit_price DOUBLE" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE ProductCategories (" + + " category_id STRING PRIMARY KEY NOT ENFORCED," + + " category_name STRING," + + " parent_category STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.verifyRelPlan( + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " pc.category_name,\n" + + " COUNT(DISTINCT o.order_id) as order_count,\n" + + " SUM(oi.quantity) as total_items,\n" + + " SUM(oi.quantity * oi.unit_price) as total_value,\n" + + " AVG(oi.unit_price) as avg_item_price,\n" + + " MAX(p.price) as max_payment,\n" + + " MIN(p.price) as min_payment,\n" + + " COUNT(CASE WHEN oi.quantity > 5 THEN 1 END) as bulk_orders\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN OrderItems oi\n" + + " ON o.order_id = oi.order_id\n" + + "LEFT JOIN ProductCategories pc\n" + + " ON oi.product_name = pc.category_id\n" + + "LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + + "GROUP BY u.user_id, u.name, pc.category_name\n" + + "HAVING COUNT(DISTINCT o.order_id) > 0"); + } + + @Test + @Tag("no-common-join-key") + void testFunctionAndExpressionWithMultiJoinNoCommonJoinKey() { + util.tableEnv() + .executeSql( + "CREATE TABLE ProductDetails (" + + " product_id STRING PRIMARY KEY NOT ENFORCED," + + " product_name STRING," + + " description STRING," + + " created_date BIGINT," + + " tags STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE UserPreferences (" + + " user_id STRING PRIMARY KEY NOT ENFORCED," + + " preferred_category STRING," + + " notification_level STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.verifyRelPlan( + "\nSELECT\n" + + " u.user_id,\n" + + " UPPER(u.name) as user_name_upper,\n" + + " LOWER(o.product) as product_lower,\n" + + " CONCAT(u.name, ' - ', o.product) as user_product,\n" + + " SUBSTRING(pd.description, 1, 50) as description_preview,\n" + + " CHAR_LENGTH(pd.description) as description_length,\n" + + " FLOOR(p.price / 100.0) * 100 as price_rounded,\n" + + " CASE\n" + + " WHEN p.price > 1000 THEN 'High'\n" + + " WHEN p.price > 500 THEN 'Medium'\n" + + " ELSE 'Low'\n" + + " END as price_tier,\n" + + " REGEXP_REPLACE(pd.tags, ',', ' | ') as formatted_tags,\n" + + " TO_TIMESTAMP_LTZ(pd.created_date, 3) as product_created,\n" + + " COALESCE(up.preferred_category, 'None') as user_preference,\n" + + " CASE\n" + + " WHEN up.notification_level = 'HIGH' THEN 'Frequent Updates'\n" + + " WHEN up.notification_level = 'MEDIUM' THEN 'Daily Updates'\n" + + " ELSE 'Weekly Updates'\n" + + " END as notification_frequency\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + + "LEFT JOIN ProductDetails pd\n" + + " ON o.product = pd.product_id\n" + + "LEFT JOIN UserPreferences up\n" + + " ON u.user_id = up.user_id"); + } + + /* + * Calcite automatically generates LogicalProject nodes for nested field access. + * As a result, each join input in this test is wrapped in a projection, which prevents + * the planner from fusing all joins into a single MultiJoin node initially. + * Therefore, in this test, each Join is still converted to a MultiJoin individually. + */ + @Test + @Tag("multijoin-chain-expected") + void testJoinConditionHasNestedFieldsMultiJoinChainExpected() { + util.tableEnv() + .executeSql( + "CREATE TABLE Developers (" + + " developer_id STRING PRIMARY KEY NOT ENFORCED," + + " person ROW>," + + " experience_years INT" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE SupportTickets (" + + " ticket_id STRING PRIMARY KEY NOT ENFORCED," + + " reporter ROW>," + + " issue STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE Feedback (" + + " feedback_id STRING PRIMARY KEY NOT ENFORCED," + + " author ROW>," + + " message STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE Subscriptions (" + + " sub_id STRING PRIMARY KEY NOT ENFORCED," + + " subscriber ROW>," + + " active BOOLEAN" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.verifyRelPlan( + "\nSELECT\n" + + " d.developer_id,\n" + + " d.person.info.name AS developer_name,\n" + + " s.ticket_id,\n" + + " s.reporter.info.priority AS ticket_priority,\n" + + " f.feedback_id,\n" + + " f.author.info.rating AS feedback_rating,\n" + + " sub.sub_id,\n" + + " sub.subscriber.info.plan AS subscription_plan\n" + + "FROM Developers AS d\n" + + "LEFT JOIN SupportTickets AS s\n" + + " ON d.person.info.id = s.reporter.info.id\n" + + "LEFT JOIN Feedback AS f\n" + + " ON d.person.info.id = f.author.info.id\n" + + "LEFT JOIN Subscriptions AS sub\n" + + " ON d.person.info.id = sub.subscriber.info.id"); + } + + @Test + @Tag("multijoin-chain-expected") + void testComplexNestedCTEWithAggregationAndFunctionsMultiJoinChainExpected() { + util.tableEnv() + .executeSql( + "CREATE TABLE OrderMetrics (" + + " metric_id STRING PRIMARY KEY NOT ENFORCED," + + " order_id STRING," + + " metric_type STRING," + + " metric_value DOUBLE" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.verifyRelPlan( + "\nWITH base_orders AS (\n" + + " SELECT u.user_id, u.name, o.order_id, p.payment_id, p.price\n" + + " FROM Users u\n" + + " INNER JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + " INNER JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + + "),\n" + + "enriched_orders AS (\n" + + " SELECT\n" + + " bo.*,\n" + + " om.metric_type,\n" + + " om.metric_value,\n" + + " CASE\n" + + " WHEN bo.price > 1000 THEN 'Premium'\n" + + " WHEN bo.price > 500 THEN 'Standard'\n" + + " ELSE 'Basic'\n" + + " END as order_tier\n" + + " FROM base_orders bo\n" + + " LEFT JOIN OrderMetrics om\n" + + " ON bo.order_id = om.order_id\n" + + "),\n" + + "aggregated_metrics AS (\n" + + " SELECT\n" + + " user_id,\n" + + " name,\n" + + " COUNT(DISTINCT order_id) as total_orders,\n" + + " SUM(price) as total_spent,\n" + + " AVG(price) as avg_order_value,\n" + + " MAX(metric_value) as max_metric,\n" + + " MIN(metric_value) as min_metric,\n" + + " COUNT(CASE WHEN order_tier = 'Premium' THEN 1 END) as premium_orders\n" + + " FROM enriched_orders\n" + + " GROUP BY user_id, name\n" + + ")\n" + + "SELECT\n" + + " user_id,\n" + + " UPPER(name) as user_name,\n" + + " total_orders,\n" + + " ROUND(total_spent, 2) as total_spent_rounded,\n" + + " ROUND(avg_order_value, 2) as avg_order_value_rounded,\n" + + " CONCAT('User: ', name, ' has ', CAST(total_orders AS STRING), ' orders') as summary,\n" + + " CASE\n" + + " WHEN total_orders > 10 THEN 'Frequent Customer'\n" + + " WHEN total_orders > 5 THEN 'Regular Customer'\n" + + " ELSE 'Occasional Customer'\n" + + " END as customer_type\n" + + "FROM aggregated_metrics\n" + + "WHERE total_spent > 0"); + } + + @Test + void testJoinOfProjections() { + util.verifyRelPlan( + "\nSELECT u.user_id, o.order_id, o.product, p.price, s.location\n" + + "FROM (SELECT user_id, name, cash FROM Users WHERE cash > 100) AS u\n" + + "JOIN (SELECT user_id, order_id, product FROM Orders WHERE product IS NOT NULL) AS o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN (SELECT user_id, price FROM Payments WHERE price > 50) AS p\n" + + " ON u.user_id = p.user_id\n" + + "LEFT JOIN (SELECT user_id, location FROM Shipments WHERE location IS NOT NULL) AS s\n" + + " ON u.user_id = s.user_id"); + } + + @Test + @Tag("multijoin-chain-expected") + void testJoinWithNestedSubqueryMultiJoinChainExpected() { + util.verifyRelPlan( + "\nSELECT *\n" + + "FROM Users u\n" + + "JOIN (\n" + + " SELECT o.user_id, o.order_id, p.payment_id, p.price\n" + + " FROM Orders o\n" + + " JOIN (\n" + + " SELECT payment_id, user_id, price\n" + + " FROM Payments\n" + + " WHERE price > 100\n" + + " ) AS p\n" + + " ON o.user_id = p.user_id\n" + + ") AS op\n" + + "ON u.user_id = op.user_id"); + } + + @Test + void testCTEWithMultiJoinV2() { + util.tableEnv() + .executeSql( + "CREATE TABLE Departments (" + + " dept_id STRING PRIMARY KEY NOT ENFORCED," + + " dept_name STRING," + + " budget DOUBLE" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE Projects (" + + " project_id STRING PRIMARY KEY NOT ENFORCED," + + " project_name STRING," + + " dept_id STRING," + + " status STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.verifyRelPlan( + "\nWITH high_budget_depts AS (\n" + + " SELECT dept_id, dept_name, budget\n" + + " FROM Departments\n" + + " WHERE budget > 600000\n" + + "),\n" + + "active_projects AS (\n" + + " SELECT project_id, project_name, dept_id\n" + + " FROM Projects\n" + + " WHERE status = 'ACTIVE'\n" + + ")\n" + + "SELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " hbd.dept_name,\n" + + " ap.project_name,\n" + + " hbd.budget\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN high_budget_depts hbd\n" + + " ON o.user_id = hbd.dept_id\n" + + "LEFT JOIN active_projects ap\n" + + " ON hbd.dept_id = ap.dept_id"); + } + + @Test + void testWithOrInJoinCondition() { + util.verifyRelPlan( + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id,\n" + + " s.location\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON o.user_id = u.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id OR u.name = p.payment_id\n" + + "LEFT JOIN Shipments s\n" + + " ON p.user_id = s.user_id"); + } + + @Test + @Tag("multijoin-chain-expected") + void testWithCastCommonJoinKeyToIntegerMultiJoinChainExpected() { + util.verifyRelPlan( + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id,\n" + + " s.location\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON o.user_id = u.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON CAST(u.user_id as INTEGER) = CAST(p.user_id as INTEGER)\n" + + "LEFT JOIN Shipments s\n" + + " ON u.user_id = s.user_id"); + } + + @Test + void testWithCastCommonJoinKeyToVarchar() { + util.verifyRelPlan( + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id,\n" + + " s.location\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON o.user_id = u.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON CAST(u.user_id as VARCHAR) = CAST(p.user_id as VARCHAR)\n" + + "LEFT JOIN Shipments s\n" + + " ON u.user_id = s.user_id"); + } + + @Test + void testAggregationAndGroupingWithMultiJoinV2() { + util.tableEnv() + .executeSql( + "CREATE TABLE Categories (" + + " category_id STRING PRIMARY KEY NOT ENFORCED," + + " category_name STRING," + + " parent_category STRING," + + " user_id STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE Sales (" + + " sale_id STRING PRIMARY KEY NOT ENFORCED," + + " user_id STRING," + + " product_id STRING," + + " amount DOUBLE," + + " sale_date DATE" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.verifyRelPlan( + "\nSELECT\n" + + " c.category_name,\n" + + " COUNT(DISTINCT u.user_id) AS unique_users,\n" + + " COUNT(s.sale_id) AS total_sales,\n" + + " SUM(s.amount) AS total_revenue,\n" + + " AVG(s.amount) AS avg_sale_amount,\n" + + " MAX(s.amount) AS max_sale_amount\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN Categories c\n" + + " ON u.user_id = c.user_id AND o.product = c.category_id\n" + + "LEFT JOIN Sales s\n" + + " ON u.user_id = s.user_id\n" + + "GROUP BY c.category_name\n" + + "HAVING COUNT(s.sale_id) > 0"); + } + + @Test + void testSameTableMultipleAliases() { + util.verifyRelPlan( + "\nSELECT *\n" + + "FROM Users u\n" + + "LEFT JOIN Users u1\n" + + " ON u.user_id = u1.user_id\n" + + "LEFT JOIN Users u2\n" + + " ON u1.user_id = u2.user_id\n" + + "LEFT JOIN Users u3\n" + + " ON u2.user_id = u3.user_id"); + } + + @Test + @Tag("multijoin-chain-expected") + void testWithExpressionInJoinConditionMultiJoinChainExpected() { + util.tableEnv() + .executeSql( + "CREATE TABLE Products (" + + " product_id STRING PRIMARY KEY NOT ENFORCED," + + " price DOUBLE," + + " discount DOUBLE" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE Sales (" + + " sale_id STRING PRIMARY KEY NOT ENFORCED," + + " product_key DOUBLE," + + " quantity INT" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE Promotions (" + + " promo_id STRING PRIMARY KEY NOT ENFORCED," + + " product_key DOUBLE," + + " promo_text STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.verifyRelPlan( + "\nSELECT\n" + + " p.product_id,\n" + + " (p.price - p.discount) AS net_price,\n" + + " s.quantity,\n" + + " pr.promo_text\n" + + "FROM Products AS p\n" + + "LEFT JOIN Sales AS s\n" + + " ON (p.price - p.discount) = s.product_key\n" + + "LEFT JOIN Promotions AS pr\n" + + " ON (p.price - p.discount) = pr.product_key\n" + + "WHERE (p.price - p.discount) > 100"); + } + + @Test + @Tag("no-common-join-key") + void testFunctionAndExpressionWithMultiJoinNoCommonJoinKeyV2() { + util.tableEnv() + .executeSql( + "CREATE TABLE ProductDetails (" + + " product_id STRING PRIMARY KEY NOT ENFORCED," + + " product_name STRING," + + " price DOUBLE," + + " weight DOUBLE," + + " created_date DATE" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE Reviews (" + + " review_id STRING PRIMARY KEY NOT ENFORCED," + + " product_id STRING," + + " rating INT," + + " review_text STRING," + + " review_date DATE" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.verifyRelPlan( + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " pd.product_name,\n" + + " pd.price,\n" + + " ROUND(pd.price * 1.1, 2) AS price_with_tax,\n" + + " CONCAT('Product: ', pd.product_name) AS product_description,\n" + + " CHAR_LENGTH(r.review_text) AS review_length,\n" + + " UPPER(SUBSTRING(r.review_text, 1, 10)) AS review_preview,\n" + + " CASE\n" + + " WHEN r.rating >= 4 THEN 'High Rating'\n" + + " WHEN r.rating >= 3 THEN 'Medium Rating'\n" + + " ELSE 'Low Rating'\n" + + " END AS rating_category,\n" + + " TIMESTAMPDIFF(DAY, pd.created_date, CURRENT_DATE) AS days_since_created\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN ProductDetails pd\n" + + " ON o.product = pd.product_id\n" + + "LEFT JOIN Reviews r\n" + + " ON pd.product_id = r.product_id"); + } + + @Test + void testCrossJoinUnnestWithMultiJoinInsert() { + util.tableEnv() + .executeSql( + "CREATE TABLE UnnestSink (" + + " detail_id STRING," + + " element_data STRING," + + " data_value_id INT," + + " user_id STRING," + + " order_id STRING" + + ") WITH ('connector' = 'values', 'sink-insert-only' = 'false')"); + + util.verifyRelPlanInsert( + "\nINSERT INTO UnnestSink\n" + + "(\n" + + " detail_id,\n" + + " element_data,\n" + + " data_value_id,\n" + + " user_id,\n" + + " order_id\n" + + ")\n" + + "SELECT\n" + + " d.detail_id,\n" + + " TRIM(REGEXP_REPLACE(edata, '[\\[\\]\\\"]', '')) AS element_data,\n" + + " ARRAY_POSITION(split(REGEXP_REPLACE(d.data, '^\\[\"|\"\\]$', '') , '\", \"'), edata) as data_value_id,\n" + + " d.user_id,\n" + + " o.order_id\n" + + "FROM Detail d\n" + + "INNER JOIN Orders o\n" + + " ON o.user_id = d.user_id\n" + + "INNER JOIN Payments p\n" + + " ON p.user_id = d.user_id\n" + + "LEFT JOIN Shipments s\n" + + " ON s.user_id = d.user_id\n" + + "CROSS JOIN UNNEST(split(REGEXP_REPLACE(d.data, '^\\[\"|\"\\]$', '') , '\", \"')) AS T(edata)\n" + + "WHERE NOT (s.location IS NOT NULL)"); + } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml index aae5d7ca42901..6c3a455d789a5 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml @@ -16,13 +16,648 @@ See the License for the specific language governing permissions and limitations under the License. --> + + + 5 THEN 1 END) as bulk_orders +FROM Users u +LEFT JOIN Orders o + ON u.user_id = o.user_id +LEFT JOIN OrderItems oi + ON o.order_id = oi.order_id +LEFT JOIN ProductCategories pc + ON oi.product_name = pc.category_id +LEFT JOIN Payments p + ON u.user_id = p.user_id +GROUP BY u.user_id, u.name, pc.category_name +HAVING COUNT(DISTINCT o.order_id) > 0]]> + + + ($3, 0)]) ++- LogicalAggregate(group=[{0, 1, 2}], order_count=[COUNT(DISTINCT $3)], total_items=[SUM($4)], total_value=[SUM($5)], avg_item_price=[AVG($6)], max_payment=[MAX($7)], min_payment=[MIN($7)], bulk_orders=[COUNT($8)]) + +- LogicalProject(user_id=[$0], name=[$1], category_name=[$12], order_id=[$3], quantity=[$9], $f5=[*($9, $10)], unit_price=[$10], price=[$15], $f8=[CASE(>($9, 5), 1, null:INTEGER)]) + +- LogicalJoin(condition=[=($0, $16)], joinType=[left]) + :- LogicalJoin(condition=[=($8, $11)], joinType=[left]) + : :- LogicalJoin(condition=[=($3, $7)], joinType=[left]) + : : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, OrderItems]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, ProductCategories]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) +]]> + + + (order_count, 0)]) ++- GroupAggregate(groupBy=[user_id, name, category_name], select=[user_id, name, category_name, COUNT_RETRACT(DISTINCT order_id) AS order_count, SUM_RETRACT(quantity) AS total_items, SUM_RETRACT($f5) AS total_value, AVG_RETRACT(unit_price) AS avg_item_price, MAX_RETRACT(price) AS max_payment, MIN_RETRACT(price) AS min_payment, COUNT_RETRACT($f8) AS bulk_orders]) + +- Exchange(distribution=[hash[user_id, name, category_name]]) + +- Calc(select=[user_id, name, category_name, order_id, quantity, *(quantity, unit_price) AS $f5, unit_price, price, CASE(>(quantity, 5), 1, null:INTEGER) AS $f8]) + +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, noUniqueKey], joinConditions=[=(user_id, user_id1)], select=[user_id,name,order_id,quantity,unit_price,category_name,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, INTEGER quantity, DOUBLE unit_price, VARCHAR(2147483647) category_name, INTEGER price, VARCHAR(2147483647) user_id1)]) + :- Exchange(distribution=[hash[user_id]]) + : +- Calc(select=[user_id, name, order_id, quantity, unit_price, category_name]) + : +- MultiJoin(commonJoinKey=[product_name], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, (category_id)], joinConditions=[=(product_name, category_id)], select=[user_id,name,order_id,product_name,quantity,unit_price,category_id,category_name], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) product_name, INTEGER quantity, DOUBLE unit_price, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name)]) + : :- Exchange(distribution=[hash[product_name]]) + : : +- Calc(select=[user_id, name, order_id, product_name, quantity, unit_price]) + : : +- MultiJoin(commonJoinKey=[order_id], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, noUniqueKey], joinConditions=[=(order_id, order_id0)], select=[user_id,name,order_id,order_id0,product_name,quantity,unit_price], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) order_id0, VARCHAR(2147483647) product_name, INTEGER quantity, DOUBLE unit_price)]) + : : :- Exchange(distribution=[hash[order_id]]) + : : : +- Calc(select=[user_id, name, order_id]) + : : : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[(user_id), (order_id)], joinConditions=[=(user_id, user_id0)], select=[user_id,name,order_id,user_id0], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0)]) + : : : :- Exchange(distribution=[hash[user_id]]) + : : : : +- ChangelogNormalize(key=[user_id]) + : : : : +- Exchange(distribution=[hash[user_id]]) + : : : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) + : : : +- Exchange(distribution=[hash[user_id]]) + : : : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : : +- Exchange(distribution=[hash[order_id]]) + : : +- Calc(select=[order_id, product_name, quantity, unit_price]) + : : +- ChangelogNormalize(key=[item_id]) + : : +- Exchange(distribution=[hash[item_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, OrderItems]], fields=[item_id, order_id, product_name, quantity, unit_price]) + : +- Exchange(distribution=[hash[category_id]]) + : +- ChangelogNormalize(key=[category_id]) + : +- Exchange(distribution=[hash[category_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, ProductCategories, project=[category_id, category_name], metadata=[]]], fields=[category_id, category_name]) + +- Exchange(distribution=[hash[user_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[price, user_id], metadata=[]]], fields=[price, user_id]) +]]> + + + + + 0]]> + + + ($2, 0)]) ++- LogicalAggregate(group=[{0}], unique_users=[COUNT(DISTINCT $1)], total_sales=[COUNT($2)], total_revenue=[SUM($3)], avg_sale_amount=[AVG($3)], max_sale_amount=[MAX($3)]) + +- LogicalProject(category_name=[$7], user_id=[$0], sale_id=[$10], amount=[$13]) + +- LogicalJoin(condition=[=($0, $11)], joinType=[left]) + :- LogicalJoin(condition=[AND(=($0, $9), =($5, $6))], joinType=[left]) + : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Categories]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Sales]]) +]]> + + + (total_sales, 0)]) ++- GroupAggregate(groupBy=[category_name], select=[category_name, COUNT_RETRACT(DISTINCT user_id) AS unique_users, COUNT_RETRACT(sale_id) AS total_sales, SUM_RETRACT(amount) AS total_revenue, AVG_RETRACT(amount) AS avg_sale_amount, MAX_RETRACT(amount) AS max_sale_amount]) + +- Exchange(distribution=[hash[category_name]]) + +- Calc(select=[category_name, user_id, sale_id, amount]) + +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, LEFT, LEFT], inputUniqueKeys=[(user_id), noUniqueKey, (category_id), (sale_id)], joinConditions=[=(user_id, user_id0), AND(=(user_id, user_id1), =(product, category_id)), =(user_id, user_id2)], select=[user_id,user_id0,product,category_id,category_name,user_id1,sale_id,user_id2,amount], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) sale_id, VARCHAR(2147483647) user_id2, DOUBLE amount)]) + :- Exchange(distribution=[hash[user_id]]) + : +- ChangelogNormalize(key=[user_id]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id], metadata=[]]], fields=[user_id]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[user_id, product], metadata=[]]], fields=[user_id, product]) + :- Exchange(distribution=[hash[user_id]]) + : +- ChangelogNormalize(key=[category_id]) + : +- Exchange(distribution=[hash[category_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Categories, project=[category_id, category_name, user_id], metadata=[]]], fields=[category_id, category_name, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- ChangelogNormalize(key=[sale_id]) + +- Exchange(distribution=[hash[sale_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Sales, project=[sale_id, user_id, amount], metadata=[]]], fields=[sale_id, user_id, amount]) +]]> + + + + + 600000 +), +active_projects AS ( + SELECT project_id, project_name, dept_id + FROM Projects + WHERE status = 'ACTIVE' +) +SELECT + u.user_id, + u.name, + o.order_id, + hbd.dept_name, + ap.project_name, + hbd.budget +FROM Users u +LEFT JOIN Orders o + ON u.user_id = o.user_id +LEFT JOIN high_budget_depts hbd + ON o.user_id = hbd.dept_id +LEFT JOIN active_projects ap + ON hbd.dept_id = ap.dept_id]]> + + + ($2, 600000)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Departments]]) + +- LogicalProject(project_id=[$0], project_name=[$1], dept_id=[$2]) + +- LogicalFilter(condition=[=($3, _UTF-16LE'ACTIVE')]) + +- LogicalTableScan(table=[[default_catalog, default_database, Projects]]) +]]> + + + (budget, 600000)]) + : +- Exchange(distribution=[hash[dept_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Departments, filter=[]]], fields=[dept_id, dept_name, budget]) + +- Exchange(distribution=[hash[dept_id]]) + +- Calc(select=[project_name, dept_id]) + +- ChangelogNormalize(key=[project_id], condition=[=(status, 'ACTIVE')]) + +- Exchange(distribution=[hash[project_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Projects, filter=[]]], fields=[project_id, project_name, dept_id, status]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 1000 THEN 'High-Value Premium' + WHEN pc.is_premium = true THEN 'Premium' + WHEN p.price > 500 THEN 'Standard High-Value' + ELSE 'Standard' + END AS product_tier, + CASE + WHEN pr.rating >= 4 AND pr.is_verified = true THEN 'Highly Recommended' + WHEN pr.rating >= 3 THEN 'Recommended' + WHEN pr.rating >= 2 THEN 'Average' + ELSE 'Not Recommended' + END AS recommendation_status, + CASE + WHEN pc.discount_rate > 0.2 THEN p.price * (1 - pc.discount_rate) + ELSE p.price + END AS final_price +FROM Users u +LEFT JOIN Orders o + ON u.user_id = o.user_id +LEFT JOIN Payments p + ON u.user_id = p.user_id +LEFT JOIN ProductCategories pc + ON o.product = pc.category_id +LEFT JOIN ProductReviews pr + ON o.product = pr.product_id]]> + + + ($7, 1000)), _UTF-16LE'High-Value Premium':VARCHAR(19) CHARACTER SET "UTF-16LE", $11, _UTF-16LE'Premium':VARCHAR(19) CHARACTER SET "UTF-16LE", >($7, 500), _UTF-16LE'Standard High-Value':VARCHAR(19) CHARACTER SET "UTF-16LE", _UTF-16LE'Standard':VARCHAR(19) CHARACTER SET "UTF-16LE")], recommendation_status=[CASE(AND(>=($15, 4), $16), _UTF-16LE'Highly Recommended':VARCHAR(18) CHARACTER SET "UTF-16LE", >=($15, 3), _UTF-16LE'Recommended':VARCHAR(18) CHARACTER SET "UTF-16LE", >=($15, 2), _UTF-16LE'Average':VARCHAR(18) CHARACTER SET "UTF-16LE", _UTF-16LE'Not Recommended':VARCHAR(18) CHARACTER SET "UTF-16LE")], final_price=[CASE(>($12, 0.2:DECIMAL(2, 1)), *($7, -(1, $12)), CAST($7):DOUBLE)]) ++- LogicalJoin(condition=[=($5, $14)], joinType=[left]) + :- LogicalJoin(condition=[=($5, $9)], joinType=[left]) + : :- LogicalJoin(condition=[=($0, $8)], joinType=[left]) + : : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, ProductCategories]]) + +- LogicalTableScan(table=[[default_catalog, default_database, ProductReviews]]) +]]> + + + (price, 1000)), 'High-Value Premium', is_premium, 'Premium', >(price, 500), 'Standard High-Value', 'Standard') AS product_tier, CASE(AND(>=(rating, 4), is_verified), 'Highly Recommended', >=(rating, 3), 'Recommended', >=(rating, 2), 'Average', 'Not Recommended') AS recommendation_status, CASE(>(discount_rate, 0.2), *(price, -(1, discount_rate)), CAST(price AS DOUBLE)) AS final_price]) ++- MultiJoin(commonJoinKey=[product], joinTypes=[LEFT, LEFT], inputUniqueKeys=[noUniqueKey, (category_id), noUniqueKey], joinConditions=[=(product, category_id), =(product, product_id)], select=[user_id,order_id,product,payment_id,price,category_id,category_name,is_premium,discount_rate,product_id,rating,is_verified], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, BOOLEAN is_premium, DOUBLE discount_rate, VARCHAR(2147483647) product_id, INTEGER rating, BOOLEAN is_verified)]) + :- Exchange(distribution=[hash[product]]) + : +- Calc(select=[user_id, order_id, product, payment_id, price]) + : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- ChangelogNormalize(key=[user_id]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id], metadata=[]]], fields=[user_id]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + :- Exchange(distribution=[hash[category_id]]) + : +- ChangelogNormalize(key=[category_id]) + : +- Exchange(distribution=[hash[category_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, ProductCategories]], fields=[category_id, category_name, is_premium, discount_rate]) + +- Exchange(distribution=[hash[product_id]]) + +- Calc(select=[product_id, rating, is_verified]) + +- ChangelogNormalize(key=[review_id]) + +- Exchange(distribution=[hash[review_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, ProductReviews]], fields=[review_id, product_id, rating, is_verified]) +]]> + + + + + 1000 THEN 'Premium' + WHEN bo.price > 500 THEN 'Standard' + ELSE 'Basic' + END as order_tier + FROM base_orders bo + LEFT JOIN OrderMetrics om + ON bo.order_id = om.order_id +), +aggregated_metrics AS ( + SELECT + user_id, + name, + COUNT(DISTINCT order_id) as total_orders, + SUM(price) as total_spent, + AVG(price) as avg_order_value, + MAX(metric_value) as max_metric, + MIN(metric_value) as min_metric, + COUNT(CASE WHEN order_tier = 'Premium' THEN 1 END) as premium_orders + FROM enriched_orders + GROUP BY user_id, name +) +SELECT + user_id, + UPPER(name) as user_name, + total_orders, + ROUND(total_spent, 2) as total_spent_rounded, + ROUND(avg_order_value, 2) as avg_order_value_rounded, + CONCAT('User: ', name, ' has ', CAST(total_orders AS STRING), ' orders') as summary, + CASE + WHEN total_orders > 10 THEN 'Frequent Customer' + WHEN total_orders > 5 THEN 'Regular Customer' + ELSE 'Occasional Customer' + END as customer_type +FROM aggregated_metrics +WHERE total_spent > 0]]> + + + ($2, 10), _UTF-16LE'Frequent Customer':VARCHAR(19) CHARACTER SET "UTF-16LE", >($2, 5), _UTF-16LE'Regular Customer':VARCHAR(19) CHARACTER SET "UTF-16LE", _UTF-16LE'Occasional Customer':VARCHAR(19) CHARACTER SET "UTF-16LE")]) ++- LogicalFilter(condition=[>($3, 0)]) + +- LogicalAggregate(group=[{0, 1}], total_orders=[COUNT(DISTINCT $2)], total_spent=[SUM($3)], avg_order_value=[AVG($3)], max_metric=[MAX($4)], min_metric=[MIN($4)], premium_orders=[COUNT($5)]) + +- LogicalProject(user_id=[$0], name=[$1], order_id=[$2], price=[$4], metric_value=[$6], $f5=[CASE(=($7, _UTF-16LE'Premium'), 1, null:INTEGER)]) + +- LogicalProject(user_id=[$0], name=[$1], order_id=[$2], payment_id=[$3], price=[$4], metric_type=[$7], metric_value=[$8], order_tier=[CASE(>($4, 1000), _UTF-16LE'Premium':VARCHAR(8) CHARACTER SET "UTF-16LE", >($4, 500), _UTF-16LE'Standard':VARCHAR(8) CHARACTER SET "UTF-16LE", _UTF-16LE'Basic':VARCHAR(8) CHARACTER SET "UTF-16LE")]) + +- LogicalJoin(condition=[=($2, $6)], joinType=[left]) + :- LogicalProject(user_id=[$0], name=[$1], order_id=[$3], payment_id=[$6], price=[$7]) + : +- LogicalJoin(condition=[=($0, $8)], joinType=[inner]) + : :- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) + +- LogicalTableScan(table=[[default_catalog, default_database, OrderMetrics]]) +]]> + + + (total_orders, 10), 'Frequent Customer', >(total_orders, 5), 'Regular Customer', 'Occasional Customer') AS customer_type], where=[>(total_spent, 0)]) ++- GroupAggregate(groupBy=[user_id, name], select=[user_id, name, COUNT_RETRACT(DISTINCT order_id) AS total_orders, SUM_RETRACT(price) AS total_spent, AVG_RETRACT(price) AS avg_order_value]) + +- Exchange(distribution=[hash[user_id, name]]) + +- MultiJoin(commonJoinKey=[order_id], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, noUniqueKey], joinConditions=[=(order_id, order_id0)], select=[user_id,name,order_id,price,order_id0,metric_value], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, INTEGER price, VARCHAR(2147483647) order_id0, DOUBLE metric_value)]) + :- Exchange(distribution=[hash[order_id]]) + : +- Calc(select=[user_id, name, order_id, price]) + : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[INNER, INNER], inputUniqueKeys=[(user_id), (order_id), noUniqueKey], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,name,order_id,user_id0,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, INTEGER price, VARCHAR(2147483647) user_id1)]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- ChangelogNormalize(key=[user_id]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[price, user_id], metadata=[]]], fields=[price, user_id]) + +- Exchange(distribution=[hash[order_id]]) + +- Calc(select=[order_id, metric_value]) + +- ChangelogNormalize(key=[metric_id]) + +- Exchange(distribution=[hash[metric_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, OrderMetrics, project=[order_id, metric_value, metric_id], metadata=[]]], fields=[order_id, metric_value, metric_id]) +]]> + + + + + + + + + + + + + - = p.price OR p.price < 0) LEFT JOIN Shipments s ON p.user_id_2 = s.user_id_3]]> + = p.price OR p.price < 0) +LEFT JOIN Shipments s + ON p.user_id = s.user_id]]> =($2, $7), <($7, 0)))], joinType=[inner]) : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) @@ -34,25 +669,25 @@ LogicalProject(user_id_0=[$0], name=[$1], order_id=[$3], payment_id=[$6], locati = price) OR (price < 0))), (user_id_2 = user_id_3)], select=[user_id_0,name,cash,order_id,user_id_1,payment_id,price,user_id_2,location,user_id_3], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id_2, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id_3)]) - :- Exchange(distribution=[hash[user_id_0]]) - : +- ChangelogNormalize(key=[user_id_0]) - : +- Exchange(distribution=[hash[user_id_0]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id_0, name, cash]) - :- Exchange(distribution=[hash[user_id_1]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1]) - :- Exchange(distribution=[hash[user_id_2]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id_2]) - +- Exchange(distribution=[hash[user_id_3]]) - +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id_3]) +Calc(select=[user_id, name, order_id, payment_id, location]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id), noUniqueKey], joinConditions=[(user_id = user_id0), ((user_id = user_id1) AND ((cash >= price) OR (price < 0))), (user_id1 = user_id2)], select=[user_id,name,cash,order_id,user_id0,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) + :- Exchange(distribution=[hash[user_id]]) + : +- ChangelogNormalize(key=[user_id]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id]) ]]> =($2, $7), <($7, 0)))], joinType=[inner]) : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) @@ -62,42 +697,56 @@ LogicalProject(user_id_0=[$0], name=[$1], order_id=[$3], payment_id=[$6], locati +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) == Optimized Physical Plan == -Calc(select=[user_id_0, name, order_id, payment_id, location]) -+- MultiJoin(commonJoinKey=[user_id_0], joinTypes=[LEFT, INNER, LEFT], inputUniqueKeys=[(user_id_0), (order_id), (payment_id), noUniqueKey], joinConditions=[=(user_id_0, user_id_1), AND(=(user_id_0, user_id_2), OR(>=(cash, price), <(price, 0))), =(user_id_2, user_id_3)], select=[user_id_0,name,cash,order_id,user_id_1,payment_id,price,user_id_2,location,user_id_3], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id_2, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id_3)]) - :- Exchange(distribution=[hash[user_id_0]]) - : +- ChangelogNormalize(key=[user_id_0]) - : +- Exchange(distribution=[hash[user_id_0]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id_0, name, cash]) - :- Exchange(distribution=[hash[user_id_1]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1]) - :- Exchange(distribution=[hash[user_id_2]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id_2]) - +- Exchange(distribution=[hash[user_id_3]]) - +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id_3]) +Calc(select=[user_id, name, order_id, payment_id, location]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id), noUniqueKey], joinConditions=[=(user_id, user_id0), AND(=(user_id, user_id1), OR(>=(cash, price), <(price, 0))), =(user_id1, user_id2)], select=[user_id,name,cash,order_id,user_id0,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) + :- Exchange(distribution=[hash[user_id]]) + : +- ChangelogNormalize(key=[user_id]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id]) == Optimized Execution Plan == -Calc(select=[user_id_0, name, order_id, payment_id, location]) -+- MultiJoin(commonJoinKey=[user_id_0], joinTypes=[LEFT, INNER, LEFT], inputUniqueKeys=[(user_id_0), (order_id), (payment_id), noUniqueKey], joinConditions=[(user_id_0 = user_id_1), ((user_id_0 = user_id_2) AND ((cash >= price) OR (price < 0))), (user_id_2 = user_id_3)], select=[user_id_0,name,cash,order_id,user_id_1,payment_id,price,user_id_2,location,user_id_3], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id_2, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id_3)]) - :- Exchange(distribution=[hash[user_id_0]]) - : +- ChangelogNormalize(key=[user_id_0]) - : +- Exchange(distribution=[hash[user_id_0]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id_0, name, cash]) - :- Exchange(distribution=[hash[user_id_1]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1]) - :- Exchange(distribution=[hash[user_id_2]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id_2]) - +- Exchange(distribution=[hash[user_id_3]]) - +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id_3]) +Calc(select=[user_id, name, order_id, payment_id, location]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id), noUniqueKey], joinConditions=[(user_id = user_id0), ((user_id = user_id1) AND ((cash >= price) OR (price < 0))), (user_id1 = user_id2)], select=[user_id,name,cash,order_id,user_id0,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) + :- Exchange(distribution=[hash[user_id]]) + : +- ChangelogNormalize(key=[user_id]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id]) ]]> - = p.price OR p.price < 0) LEFT JOIN Shipments s ON p.user_id_2 = s.user_id_3]]> + = p.price OR p.price < 0) +LEFT JOIN Shipments s + ON p.user_id = s.user_id]]> =($2, $7), <($7, 0)))], joinType=[inner]) : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) @@ -109,64 +758,101 @@ LogicalProject(user_id_0=[$0], name=[$1], order_id=[$3], payment_id=[$6], locati =(cash, price), <(price, 0))), =(user_id_2, user_id_3)], select=[user_id_0,name,cash,order_id,user_id_1,payment_id,price,user_id_2,location,user_id_3], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id_2, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id_3)]) - :- Exchange(distribution=[hash[user_id_0]]) - : +- ChangelogNormalize(key=[user_id_0]) - : +- Exchange(distribution=[hash[user_id_0]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id_0, name, cash]) - :- Exchange(distribution=[hash[user_id_1]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1]) - :- Exchange(distribution=[hash[user_id_2]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id_2]) - +- Exchange(distribution=[hash[user_id_3]]) - +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id_3]) +Calc(select=[user_id, name, order_id, payment_id, location]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id), noUniqueKey], joinConditions=[=(user_id, user_id0), AND(=(user_id, user_id1), OR(>=(cash, price), <(price, 0))), =(user_id1, user_id2)], select=[user_id,name,cash,order_id,user_id0,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) + :- Exchange(distribution=[hash[user_id]]) + : +- ChangelogNormalize(key=[user_id]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id]) ]]> - + - + - + + + + + + = FLOOR(p.price) OR p.price < 0) +LEFT JOIN Shipments s + ON p.payment_id = s.location]]> + + + =(FLOOR($2), FLOOR($8)), <($8, 0)))], joinType=[left]) + : :- LogicalProject(user_id=[$0], name=[$1], cash=[$2], order_id=[$3], user_id0=[$4], product=[$5], $f6=[UPPER($1)]) + : : +- LogicalJoin(condition=[=($4, $0)], joinType=[left]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : +- LogicalProject(payment_id=[$0], price=[$1], user_id=[$2], $f3=[UPPER($0)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) +]]> + + + =(FLOOR(cash), FLOOR(price)), <(price, 0)))], select=[user_id,name,cash,order_id,$f6,payment_id,price,user_id1,$f3], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) $f6, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) $f3)]) +: :- Exchange(distribution=[hash[user_id, $f6]]) +: : +- Calc(select=[user_id, name, cash, order_id, UPPER(name) AS $f6]) +: : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[(user_id), (order_id)], joinConditions=[=(user_id0, user_id)], select=[user_id,name,cash,order_id,user_id0], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0)]) +: : :- Exchange(distribution=[hash[user_id]]) +: : : +- ChangelogNormalize(key=[user_id]) +: : : +- Exchange(distribution=[hash[user_id]]) +: : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) +: : +- Exchange(distribution=[hash[user_id]]) +: : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) +: +- Exchange(distribution=[hash[user_id, $f3]]) +: +- Calc(select=[payment_id, price, user_id, UPPER(payment_id) AS $f3]) +: +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) ++- Exchange(distribution=[hash[location]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments, project=[location], metadata=[]]], fields=[location]) ]]> - + + + + + + 1000 THEN 'High' + WHEN p.price > 500 THEN 'Medium' + ELSE 'Low' + END as price_tier, + REGEXP_REPLACE(pd.tags, ',', ' | ') as formatted_tags, + TO_TIMESTAMP_LTZ(pd.created_date, 3) as product_created, + COALESCE(up.preferred_category, 'None') as user_preference, + CASE + WHEN up.notification_level = 'HIGH' THEN 'Frequent Updates' + WHEN up.notification_level = 'MEDIUM' THEN 'Daily Updates' + ELSE 'Weekly Updates' + END as notification_frequency +FROM Users u +LEFT JOIN Orders o + ON u.user_id = o.user_id +LEFT JOIN Payments p + ON u.user_id = p.user_id +LEFT JOIN ProductDetails pd + ON o.product = pd.product_id +LEFT JOIN UserPreferences up + ON u.user_id = up.user_id]]> + + + ($7, 1000), _UTF-16LE'High':VARCHAR(6) CHARACTER SET "UTF-16LE", >($7, 500), _UTF-16LE'Medium':VARCHAR(6) CHARACTER SET "UTF-16LE", _UTF-16LE'Low':VARCHAR(6) CHARACTER SET "UTF-16LE")], formatted_tags=[REGEXP_REPLACE($13, _UTF-16LE',', _UTF-16LE' | ')], product_created=[TO_TIMESTAMP_LTZ($12, 3)], user_preference=[COALESCE($15, _UTF-16LE'None')], notification_frequency=[CASE(=($16, _UTF-16LE'HIGH'), _UTF-16LE'Frequent Updates':VARCHAR(16) CHARACTER SET "UTF-16LE", =($16, _UTF-16LE'MEDIUM'), _UTF-16LE'Daily Updates':VARCHAR(16) CHARACTER SET "UTF-16LE", _UTF-16LE'Weekly Updates':VARCHAR(16) CHARACTER SET "UTF-16LE")]) ++- LogicalJoin(condition=[=($0, $14)], joinType=[left]) + :- LogicalJoin(condition=[=($5, $9)], joinType=[left]) + : :- LogicalJoin(condition=[=($0, $8)], joinType=[left]) + : : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, ProductDetails]]) + +- LogicalTableScan(table=[[default_catalog, default_database, UserPreferences]]) +]]> + + + (price, 1000), 'High', >(price, 500), 'Medium', 'Low') AS price_tier, REGEXP_REPLACE(tags, ',', ' | ') AS formatted_tags, TO_TIMESTAMP_LTZ(created_date, 3) AS product_created, COALESCE(preferred_category, 'None') AS user_preference, CASE(=(notification_level, 'HIGH'), 'Frequent Updates', =(notification_level, 'MEDIUM'), 'Daily Updates', 'Weekly Updates') AS notification_frequency]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, (user_id)], joinConditions=[=(user_id, user_id2)], select=[user_id,name,product,price,description,created_date,tags,user_id2,preferred_category,notification_level], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) product, INTEGER price, VARCHAR(2147483647) description, BIGINT created_date, VARCHAR(2147483647) tags, VARCHAR(2147483647) user_id2, VARCHAR(2147483647) preferred_category, VARCHAR(2147483647) notification_level)]) + :- Exchange(distribution=[hash[user_id]]) + : +- Calc(select=[user_id, name, product, price, description, created_date, tags]) + : +- MultiJoin(commonJoinKey=[product], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, (product_id)], joinConditions=[=(product, product_id)], select=[user_id,name,product,price,product_id,description,created_date,tags], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) product, INTEGER price, VARCHAR(2147483647) product_id, VARCHAR(2147483647) description, BIGINT created_date, VARCHAR(2147483647) tags)]) + : :- Exchange(distribution=[hash[product]]) + : : +- Calc(select=[user_id, name, product, price]) + : : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, LEFT], inputUniqueKeys=[(user_id), noUniqueKey, noUniqueKey], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,name,user_id0,product,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, INTEGER price, VARCHAR(2147483647) user_id1)]) + : : :- Exchange(distribution=[hash[user_id]]) + : : : +- ChangelogNormalize(key=[user_id]) + : : : +- Exchange(distribution=[hash[user_id]]) + : : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) + : : :- Exchange(distribution=[hash[user_id]]) + : : : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[user_id, product], metadata=[]]], fields=[user_id, product]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[price, user_id], metadata=[]]], fields=[price, user_id]) + : +- Exchange(distribution=[hash[product_id]]) + : +- ChangelogNormalize(key=[product_id]) + : +- Exchange(distribution=[hash[product_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, ProductDetails, project=[product_id, description, created_date, tags], metadata=[]]], fields=[product_id, description, created_date, tags]) + +- Exchange(distribution=[hash[user_id]]) + +- ChangelogNormalize(key=[user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, UserPreferences]], fields=[user_id, preferred_category, notification_level]) +]]> + + + + + = 4 THEN 'High Rating' + WHEN r.rating >= 3 THEN 'Medium Rating' + ELSE 'Low Rating' + END AS rating_category, + TIMESTAMPDIFF(DAY, pd.created_date, CURRENT_DATE) AS days_since_created +FROM Users u +LEFT JOIN Orders o + ON u.user_id = o.user_id +LEFT JOIN ProductDetails pd + ON o.product = pd.product_id +LEFT JOIN Reviews r + ON pd.product_id = r.product_id]]> + + + =($13, 4), _UTF-16LE'High Rating':VARCHAR(13) CHARACTER SET "UTF-16LE", >=($13, 3), _UTF-16LE'Medium Rating':VARCHAR(13) CHARACTER SET "UTF-16LE", _UTF-16LE'Low Rating':VARCHAR(13) CHARACTER SET "UTF-16LE")], days_since_created=[CAST(/INT(Reinterpret(-(CURRENT_DATE, $10)), 86400000)):INTEGER]) ++- LogicalJoin(condition=[=($6, $12)], joinType=[left]) + :- LogicalJoin(condition=[=($5, $6)], joinType=[left]) + : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, ProductDetails]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Reviews]]) +]]> + + + =(rating, 4), 'High Rating', >=(rating, 3), 'Medium Rating', 'Low Rating') AS rating_category, CAST(/INT(Reinterpret(-(CURRENT_DATE(), created_date)), 86400000) AS INTEGER) AS days_since_created]) ++- MultiJoin(commonJoinKey=[product], joinTypes=[LEFT, LEFT], inputUniqueKeys=[noUniqueKey, (product_id), noUniqueKey], joinConditions=[=(product, product_id), =(product_id, product_id0)], select=[user_id,name,order_id,product,product_id,product_name,price,created_date,product_id0,rating,review_text], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) product, VARCHAR(2147483647) product_id, VARCHAR(2147483647) product_name, DOUBLE price, DATE created_date, VARCHAR(2147483647) product_id0, INTEGER rating, VARCHAR(2147483647) review_text)]) + :- Exchange(distribution=[hash[product]]) + : +- Calc(select=[user_id, name, order_id, product]) + : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[(user_id), (order_id)], joinConditions=[=(user_id, user_id0)], select=[user_id,name,order_id,user_id0,product], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product)]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- ChangelogNormalize(key=[user_id]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + :- Exchange(distribution=[hash[product_id]]) + : +- ChangelogNormalize(key=[product_id]) + : +- Exchange(distribution=[hash[product_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, ProductDetails, project=[product_id, product_name, price, created_date], metadata=[]]], fields=[product_id, product_name, price, created_date]) + +- Exchange(distribution=[hash[product_id]]) + +- Calc(select=[product_id, rating, review_text]) + +- ChangelogNormalize(key=[review_id]) + +- Exchange(distribution=[hash[review_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Reviews, project=[product_id, rating, review_text, review_id], metadata=[]]], fields=[product_id, rating, review_text, review_id]) ]]> - + =($2, -($5, 60000:INTERVAL MINUTE)), <=($2, +($5, 60000:INTERVAL MINUTE)))], joinType=[inner]) - :- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($2, 5000:INTERVAL SECOND)]) + :- LogicalWatermarkAssigner(rowtime=[$rowtime], watermark=[-($2, 5000:INTERVAL SECOND)]) : +- LogicalTableScan(table=[[default_catalog, default_database, EventTable1]]) - +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($2, 5000:INTERVAL SECOND)]) + +- LogicalWatermarkAssigner(rowtime=[$rowtime], watermark=[-($2, 5000:INTERVAL SECOND)]) +- LogicalTableScan(table=[[default_catalog, default_database, EventTable2]]) ]]> =(rowtime, -(rowtime0, 60000:INTERVAL MINUTE)), <=(rowtime, +(rowtime0, 60000:INTERVAL MINUTE)))], select=[id, val, rowtime, id0, price, rowtime0]) ++- IntervalJoin(joinType=[InnerJoin], windowBounds=[isRowTime=true, leftLowerBound=-60000, leftUpperBound=60000, leftTimeIndex=2, rightTimeIndex=2], where=[AND(=(id, id0), >=($rowtime, -($rowtime0, 60000:INTERVAL MINUTE)), <=($rowtime, +($rowtime0, 60000:INTERVAL MINUTE)))], select=[id, val, $rowtime, id0, price, $rowtime0]) :- Exchange(distribution=[hash[id]]) - : +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 5000:INTERVAL SECOND)]) - : +- TableSourceScan(table=[[default_catalog, default_database, EventTable1]], fields=[id, val, rowtime]) + : +- WatermarkAssigner(rowtime=[$rowtime], watermark=[-($rowtime, 5000:INTERVAL SECOND)]) + : +- TableSourceScan(table=[[default_catalog, default_database, EventTable1]], fields=[id, val, $rowtime]) +- Exchange(distribution=[hash[id]]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 5000:INTERVAL SECOND)]) - +- TableSourceScan(table=[[default_catalog, default_database, EventTable2]], fields=[id, price, rowtime]) + +- WatermarkAssigner(rowtime=[$rowtime], watermark=[-($rowtime, 5000:INTERVAL SECOND)]) + +- TableSourceScan(table=[[default_catalog, default_database, EventTable2]], fields=[id, price, $rowtime]) +]]> + + + + + + + + + + + + + + + + 100) AS u +JOIN (SELECT user_id, order_id, product FROM Orders WHERE product IS NOT NULL) AS o + ON u.user_id = o.user_id +LEFT JOIN (SELECT user_id, price FROM Payments WHERE price > 50) AS p + ON u.user_id = p.user_id +LEFT JOIN (SELECT user_id, location FROM Shipments WHERE location IS NOT NULL) AS s + ON u.user_id = s.user_id]]> + + + ($2, 100)]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : +- LogicalProject(user_id=[$1], order_id=[$0], product=[$2]) + : : +- LogicalFilter(condition=[IS NOT NULL($2)]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : +- LogicalProject(user_id=[$2], price=[$1]) + : +- LogicalFilter(condition=[>($1, 50)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) + +- LogicalProject(user_id=[$1], location=[$0]) + +- LogicalFilter(condition=[IS NOT NULL($0)]) + +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) +]]> + + + (cash, 100)]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[], project=[user_id, cash], metadata=[]]], fields=[user_id, cash]) + :- Exchange(distribution=[hash[user_id]]) + : +- Calc(select=[user_id, order_id, product], where=[IS NOT NULL(product)]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, filter=[]]], fields=[order_id, user_id, product]) + :- Exchange(distribution=[hash[user_id]]) + : +- Calc(select=[user_id, price], where=[>(price, 50)]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[], project=[price, user_id], metadata=[]]], fields=[price, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- Calc(select=[user_id, location], where=[IS NOT NULL(location)]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments, filter=[]]], fields=[location, user_id]) +]]> + + + + + 100 + ) AS p + ON o.user_id = p.user_id +) AS op +ON u.user_id = op.user_id]]> + + + ($1, 100)]) + +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) +]]> + + + (price, 100)]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) ]]> @@ -288,7 +1364,26 @@ Calc(select=[a, DATE_FORMAT(CURRENT_TIMESTAMP(), _UTF-16LE'yyMMdd') AS day], cha - = 0 JOIN AddressPK a ON u.user_id = a.user_id AND a.location IS NOT NULL]]> + = 0 +JOIN AddressPK a + ON u.user_id = a.user_id + AND a.location IS NOT NULL]]> - + - + - + - + - + - + + + + + + + + + + + + - + - + - + - + - + - + - + - + - + - + - + - + - + - 10]]> + 10]]> ($7, 10))]) +- LogicalJoin(condition=[=($0, $8)], joinType=[left]) :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) @@ -861,25 +2166,25 @@ LogicalProject(user_id_0=[$0], name=[$1], order_id=[$3], payment_id=[$6]) 10)]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id_2]) +Calc(select=[user_id, CAST('Gus' AS VARCHAR(2147483647)) AS name, order_id, CAST(payment_id AS VARCHAR(2147483647)) AS payment_id]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[(user_id = user_id0), (user_id = user_id1)], select=[user_id,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) + :- Exchange(distribution=[hash[user_id]]) + : +- Calc(select=[user_id]) + : +- ChangelogNormalize(key=[user_id], condition=[(name = 'Gus')]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[], project=[user_id, name], metadata=[]]], fields=[user_id, name]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- Calc(select=[payment_id, user_id], where=[(price > 10)]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) ]]> ($7, 10))]) +- LogicalJoin(condition=[=($0, $8)], joinType=[left]) :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) @@ -888,42 +2193,53 @@ LogicalProject(user_id_0=[$0], name=[$1], order_id=[$3], payment_id=[$6]) +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) == Optimized Physical Plan == -Calc(select=[user_id_0, CAST(_UTF-16LE'Gus':VARCHAR(2147483647) CHARACTER SET "UTF-16LE" AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE") AS name, order_id, CAST(payment_id AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE") AS payment_id]) -+- MultiJoin(commonJoinKey=[user_id_0], joinTypes=[LEFT, INNER], inputUniqueKeys=[(user_id_0), (order_id), (payment_id)], joinConditions=[=(user_id_0, user_id_1), =(user_id_0, user_id_2)], select=[user_id_0,order_id,user_id_1,payment_id,user_id_2], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id_2)]) - :- Exchange(distribution=[hash[user_id_0]]) - : +- Calc(select=[user_id_0]) - : +- ChangelogNormalize(key=[user_id_0], condition=[=(name, _UTF-16LE'Gus':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")]) - : +- Exchange(distribution=[hash[user_id_0]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[], project=[user_id_0, name], metadata=[]]], fields=[user_id_0, name]) - :- Exchange(distribution=[hash[user_id_1]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1]) - +- Exchange(distribution=[hash[user_id_2]]) - +- Calc(select=[payment_id, user_id_2], where=[>(price, 10)]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id_2]) +Calc(select=[user_id, CAST(_UTF-16LE'Gus':VARCHAR(2147483647) CHARACTER SET "UTF-16LE" AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE") AS name, order_id, CAST(payment_id AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE") AS payment_id]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) + :- Exchange(distribution=[hash[user_id]]) + : +- Calc(select=[user_id]) + : +- ChangelogNormalize(key=[user_id], condition=[=(name, _UTF-16LE'Gus':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[], project=[user_id, name], metadata=[]]], fields=[user_id, name]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- Calc(select=[payment_id, user_id], where=[>(price, 10)]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) == Optimized Execution Plan == -Calc(select=[user_id_0, CAST('Gus' AS VARCHAR(2147483647)) AS name, order_id, CAST(payment_id AS VARCHAR(2147483647)) AS payment_id]) -+- MultiJoin(commonJoinKey=[user_id_0], joinTypes=[LEFT, INNER], inputUniqueKeys=[(user_id_0), (order_id), (payment_id)], joinConditions=[(user_id_0 = user_id_1), (user_id_0 = user_id_2)], select=[user_id_0,order_id,user_id_1,payment_id,user_id_2], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id_2)]) - :- Exchange(distribution=[hash[user_id_0]]) - : +- Calc(select=[user_id_0]) - : +- ChangelogNormalize(key=[user_id_0], condition=[(name = 'Gus')]) - : +- Exchange(distribution=[hash[user_id_0]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[], project=[user_id_0, name], metadata=[]]], fields=[user_id_0, name]) - :- Exchange(distribution=[hash[user_id_1]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1]) - +- Exchange(distribution=[hash[user_id_2]]) - +- Calc(select=[payment_id, user_id_2], where=[(price > 10)]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id_2]) +Calc(select=[user_id, CAST('Gus' AS VARCHAR(2147483647)) AS name, order_id, CAST(payment_id AS VARCHAR(2147483647)) AS payment_id]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[(user_id = user_id0), (user_id = user_id1)], select=[user_id,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) + :- Exchange(distribution=[hash[user_id]]) + : +- Calc(select=[user_id]) + : +- ChangelogNormalize(key=[user_id], condition=[(name = 'Gus')]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[], project=[user_id, name], metadata=[]]], fields=[user_id, name]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- Calc(select=[payment_id, user_id], where=[(price > 10)]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) ]]> - 10]]> + 10]]> ($7, 10))]) +- LogicalJoin(condition=[=($0, $8)], joinType=[left]) :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) @@ -934,80 +2250,310 @@ LogicalProject(user_id_0=[$0], name=[$1], order_id=[$3], payment_id=[$6]) (price, 10)]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id_2]) +Calc(select=[user_id, CAST('Gus' AS VARCHAR(2147483647)) AS name, order_id, CAST(payment_id AS VARCHAR(2147483647)) AS payment_id]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) + :- Exchange(distribution=[hash[user_id]]) + : +- Calc(select=[user_id]) + : +- ChangelogNormalize(key=[user_id], condition=[=(name, 'Gus')]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[], project=[user_id, name], metadata=[]]], fields=[user_id, name]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- Calc(select=[payment_id, user_id], where=[>(price, 10)]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) ]]> - + - + + + + + + + + + + + + + + + + + + + + + + + + + + + + 100]]> + + + (-($1, $2), 100)]) + +- LogicalProject(product_id=[$0], price=[$1], discount=[$2], sale_id=[$3], product_key=[$4], quantity=[$5], promo_id=[$7], product_key0=[$8], promo_text=[$9]) + +- LogicalJoin(condition=[=($6, $8)], joinType=[left]) + :- LogicalProject(product_id=[$0], price=[$1], discount=[$2], sale_id=[$3], product_key=[$4], quantity=[$5], $f6=[-($1, $2)]) + : +- LogicalProject(product_id=[$0], price=[$1], discount=[$2], sale_id=[$4], product_key=[$5], quantity=[$6]) + : +- LogicalJoin(condition=[=($3, $5)], joinType=[left]) + : :- LogicalProject(product_id=[$0], price=[$1], discount=[$2], $f3=[-($1, $2)]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Products]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Sales]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Promotions]]) +]]> + + + (-(price, discount), 100)]) + : : +- Exchange(distribution=[hash[product_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Products, filter=[]]], fields=[product_id, price, discount]) + : +- Exchange(distribution=[hash[product_key]]) + : +- Calc(select=[product_key, quantity]) + : +- ChangelogNormalize(key=[sale_id]) + : +- Exchange(distribution=[hash[sale_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Sales]], fields=[sale_id, product_key, quantity]) + +- Exchange(distribution=[hash[product_key]]) + +- Calc(select=[product_key, promo_text]) + +- ChangelogNormalize(key=[promo_id]) + +- Exchange(distribution=[hash[promo_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Promotions]], fields=[promo_id, product_key, promo_text]) +]]> + + + + + + + + + + + diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/keyselector/AttributeBasedJoinKeyExtractor.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/keyselector/AttributeBasedJoinKeyExtractor.java index 52dd508cd15eb..d5e073c8b9b67 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/keyselector/AttributeBasedJoinKeyExtractor.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/keyselector/AttributeBasedJoinKeyExtractor.java @@ -22,6 +22,7 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.utils.NoCommonJoinKeyException; import javax.annotation.Nullable; @@ -532,7 +533,7 @@ private void processCommonAttributes( if (commonAttrsForThisInput.isEmpty()) { // This indicates that there is no common join key among all inputs. // In this case, we cannot use a multi-join, so throw an exception. - throw new IllegalStateException( + throw new NoCommonJoinKeyException( "All inputs in a multi-way join must share a common join key. Input #" + currentInputId + " does not share a join key with the other inputs. Please ensure all join" From ad47717862567f76216e68a07b474a593382001e Mon Sep 17 00:00:00 2001 From: Sergey Nuyanzin Date: Tue, 2 Dec 2025 14:16:03 +0100 Subject: [PATCH 20/31] [FLINK-38750][table] Validation of queries with functions erroneously invoked under `SELECT` fails with `StackOverflow` --- .../inference/TypeInferenceOperandInference.java | 4 +++- .../table/planner/plan/stream/sql/CalcTest.scala | 15 +++++++++++++++ 2 files changed, 18 insertions(+), 1 deletion(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandInference.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandInference.java index fba8d45034fec..51835f1004cb9 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandInference.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandInference.java @@ -81,8 +81,10 @@ public void inferOperandTypes( false)) { inferOperandTypesOrError(unwrapTypeFactory(callBinding), callContext, operandTypes); } - } catch (ValidationException | CalciteContextException e) { + } catch (ValidationException e) { // let operand checker fail + } catch (CalciteContextException e) { + throw e; } catch (Throwable t) { throw createUnexpectedException(callContext, t); } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/CalcTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/CalcTest.scala index 45ef3e380ee33..d6c47d48459f3 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/CalcTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/CalcTest.scala @@ -108,6 +108,21 @@ class CalcTest extends TableTestBase { .isThrownBy(() => util.tableEnv.sqlQuery("SELECT a, foo FROM MyTable")) } + @Test + def testCoalesceOnInvalidField(): Unit = { + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy(() => util.verifyExecPlan("SELECT coalesce(SELECT invalid)")) + .withMessageContaining("Column 'invalid' not found in any table") + } + + @Test + def testNestedCoalesceOnInvalidField(): Unit = { + assertThatExceptionOfType(classOf[ValidationException]) + .isThrownBy( + () => util.verifyExecPlan("SELECT coalesce(SELECT coalesce(SELECT coalesce(invalid)))")) + .withMessageContaining("Column 'invalid' not found in any table") + } + @Test def testPrimitiveMapType(): Unit = { util.verifyExecPlan("SELECT MAP[b, 30, 10, a] FROM MyTable") From 3f79fac6080a77356d3f7c9e705e83955978da7e Mon Sep 17 00:00:00 2001 From: Hao Li <1127478+lihaosky@users.noreply.github.com> Date: Tue, 2 Dec 2025 18:27:45 -0800 Subject: [PATCH 21/31] [FLINK-38084][doc] Add download doc for model providers (#27296) --- .../docs/connectors/models/downloads.md | 41 ++++++++++++ .../docs/connectors/models/downloads.md | 42 ++++++++++++ docs/data/sql_models.yml | 22 ++++++ .../shortcodes/sql_optional_models.html | 67 +++++++++++++++++++ 4 files changed, 172 insertions(+) create mode 100644 docs/content.zh/docs/connectors/models/downloads.md create mode 100644 docs/content/docs/connectors/models/downloads.md create mode 100644 docs/data/sql_models.yml create mode 100644 docs/layouts/shortcodes/sql_optional_models.html diff --git a/docs/content.zh/docs/connectors/models/downloads.md b/docs/content.zh/docs/connectors/models/downloads.md new file mode 100644 index 0000000000000..0eb85a304aa0e --- /dev/null +++ b/docs/content.zh/docs/connectors/models/downloads.md @@ -0,0 +1,41 @@ +--- +title: 下载页面 +weight: 100 +type: docs +bookToc: false +aliases: + - /zh/dev/table/connectors/downloads.html +--- + + +# SQL Models 下载页面 + +{{< unstable >}} +{{< hint info >}} +Download links are available only for stable releases. +{{< /hint >}} +{{< /unstable >}} + +The page contains links to optional sql-client models that are not part of the binary distribution. + +# 可选的 SQL Models +------------------- + +{{< sql_optional_models >}} diff --git a/docs/content/docs/connectors/models/downloads.md b/docs/content/docs/connectors/models/downloads.md new file mode 100644 index 0000000000000..6b5119f333acb --- /dev/null +++ b/docs/content/docs/connectors/models/downloads.md @@ -0,0 +1,42 @@ +--- +title: Download +weight: 100 +type: docs +bookToc: false +aliases: + - /dev/table/models/downloads.html +--- + + +# SQL Models download page + +{{< unstable >}} +{{< hint info >}} +Download links are available only for stable releases. +{{< /hint >}} +{{< /unstable >}} + +The page contains links to optional SQL Client models that are not part of the binary distribution. + +# Optional SQL models +------------------- + +{{< sql_optional_models >}} + diff --git a/docs/data/sql_models.yml b/docs/data/sql_models.yml new file mode 100644 index 0000000000000..74cd32b1e870e --- /dev/null +++ b/docs/data/sql_models.yml @@ -0,0 +1,22 @@ +# 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 + +openai: + name: OpenAI + category: model + maven: flink-model-openai + sql_url: https://repo.maven.apache.org/maven2/org/apache/flink/flink-model-openai/$version/flink-model-openai-$version.jar diff --git a/docs/layouts/shortcodes/sql_optional_models.html b/docs/layouts/shortcodes/sql_optional_models.html new file mode 100644 index 0000000000000..ad93dbc9814eb --- /dev/null +++ b/docs/layouts/shortcodes/sql_optional_models.html @@ -0,0 +1,67 @@ +{{/* +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. +*/}}{{/* + Generates the SQL download model table. +*/}} + + + + + + + + + + + {{ range .Site.Data.sql_models }} + {{ if eq .category "model" }}{{ if eq .builtin nil }} + {{ if eq .versions nil }} + + + + + {{ else }} + Only available for stable versions. + {{ end }} + + {{ else }} + {{ $name := .name }} + {{ range .versions }} + + + + + {{ else }} + Only available for stable versions. + {{ end }} + + {{ end }} + {{ end }} + {{ end }}{{ end }} + {{ end }} + +
NameVersionDownload Link
{{- .name -}} + {{ if $.Site.Params.IsStable }} + Download + (asc, + sha1)
{{- $name -}}{{- .version -}} + {{ if $.Site.Params.IsStable }} + Download + (asc, + sha1)
+ From 3546641020afb5563ef708dd5fc288cfd9f80f7d Mon Sep 17 00:00:00 2001 From: Hao Li <1127478+lihaosky@users.noreply.github.com> Date: Wed, 3 Dec 2025 17:35:08 -0800 Subject: [PATCH 22/31] [FLINK-38767][table] Fix vector search execnode transform name (#27304) --- .../nodes/exec/batch/BatchExecVectorSearchTableFunction.java | 3 +-- .../nodes/exec/stream/StreamExecVectorSearchTableFunction.java | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecVectorSearchTableFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecVectorSearchTableFunction.java index 51f32a685c50c..6fcbc4ec4b536 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecVectorSearchTableFunction.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecVectorSearchTableFunction.java @@ -28,7 +28,6 @@ import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecVectorSearchTableFunction; import org.apache.flink.table.planner.plan.nodes.exec.spec.VectorSearchSpec; import org.apache.flink.table.planner.plan.nodes.exec.spec.VectorSearchTableSourceSpec; -import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecMLPredictTableFunction; import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecVectorSearchTableFunction; import org.apache.flink.table.planner.plan.utils.FunctionCallUtil; import org.apache.flink.table.types.logical.RowType; @@ -50,7 +49,7 @@ "table.exec.async-vector-search.timeout", "table.exec.async-vector-search.output-mode" }, - producedTransformations = StreamExecMLPredictTableFunction.ML_PREDICT_TRANSFORMATION, + producedTransformations = CommonExecVectorSearchTableFunction.VECTOR_SEARCH_TRANSFORMATION, minPlanVersion = FlinkVersion.v2_2, minStateVersion = FlinkVersion.v2_2) public class BatchExecVectorSearchTableFunction extends CommonExecVectorSearchTableFunction diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecVectorSearchTableFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecVectorSearchTableFunction.java index 101a58dabdeca..849a6b68788e6 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecVectorSearchTableFunction.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecVectorSearchTableFunction.java @@ -49,7 +49,7 @@ "table.exec.async-vector-search.timeout", "table.exec.async-vector-search.output-mode" }, - producedTransformations = StreamExecMLPredictTableFunction.ML_PREDICT_TRANSFORMATION, + producedTransformations = CommonExecVectorSearchTableFunction.VECTOR_SEARCH_TRANSFORMATION, minPlanVersion = FlinkVersion.v2_2, minStateVersion = FlinkVersion.v2_2) public class StreamExecVectorSearchTableFunction extends CommonExecVectorSearchTableFunction From 8de272c7d9627b77d9a9b5693e5b1dff6ee4ddb0 Mon Sep 17 00:00:00 2001 From: Hao Li <1127478+lihaosky@users.noreply.github.com> Date: Thu, 4 Dec 2025 03:15:48 -0800 Subject: [PATCH 23/31] [FLINK-38773][table] Fix batch vector search excnode context (#27311) --- .../nodes/exec/batch/BatchExecVectorSearchTableFunction.java | 5 ++--- .../flink/table/planner/plan/utils/ExecNodeMetadataUtil.java | 2 ++ .../async-vector-search/plan/async-vector-search.json | 4 ++-- .../sync-vector-search/plan/sync-vector-search.json | 4 ++-- .../plan/vector-search-with-runtime-config.json | 4 ++-- 5 files changed, 10 insertions(+), 9 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecVectorSearchTableFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecVectorSearchTableFunction.java index 6fcbc4ec4b536..204da67ac2ed4 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecVectorSearchTableFunction.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecVectorSearchTableFunction.java @@ -28,7 +28,6 @@ import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecVectorSearchTableFunction; import org.apache.flink.table.planner.plan.nodes.exec.spec.VectorSearchSpec; import org.apache.flink.table.planner.plan.nodes.exec.spec.VectorSearchTableSourceSpec; -import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecVectorSearchTableFunction; import org.apache.flink.table.planner.plan.utils.FunctionCallUtil; import org.apache.flink.table.types.logical.RowType; @@ -65,9 +64,9 @@ public BatchExecVectorSearchTableFunction( String description) { this( ExecNodeContext.newNodeId(), - ExecNodeContext.newContext(StreamExecVectorSearchTableFunction.class), + ExecNodeContext.newContext(BatchExecVectorSearchTableFunction.class), ExecNodeContext.newPersistedConfig( - StreamExecVectorSearchTableFunction.class, tableConfig), + BatchExecVectorSearchTableFunction.class, tableConfig), tableSourceSpec, vectorSearchSpec, asyncOptions, 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 cd24cc9c82b55..2aaeb5e8ea087 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 @@ -47,6 +47,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecTableSourceScan; import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecUnion; import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecValues; +import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecVectorSearchTableFunction; import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecWindowTableFunction; import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecAsyncCalc; import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecAsyncCorrelate; @@ -201,6 +202,7 @@ private ExecNodeMetadataUtil() { add(BatchExecMatch.class); add(BatchExecOverAggregate.class); add(BatchExecRank.class); + add(BatchExecVectorSearchTableFunction.class); } }; diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/batch-exec-vector-search-table-function_1/async-vector-search/plan/async-vector-search.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/batch-exec-vector-search-table-function_1/async-vector-search/plan/async-vector-search.json index 1a98b09d924e2..840a4850a95af 100644 --- a/flink-table/flink-table-planner/src/test/resources/restore-tests/batch-exec-vector-search-table-function_1/async-vector-search/plan/async-vector-search.json +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/batch-exec-vector-search-table-function_1/async-vector-search/plan/async-vector-search.json @@ -24,10 +24,10 @@ }, "outputType" : "ROW<`id` BIGINT, `content` VARCHAR(2147483647), `vector` ARRAY>", "description" : "TableSourceScan(table=[[default_catalog, default_database, src_t]], fields=[id, content, vector])", - "dynamicFilteringDataListenerID" : "581f0987-928f-4c00-a1c1-500fdd1c98fb" + "dynamicFilteringDataListenerID" : "1753d1db-2d8a-4ca8-9d85-18ec5670a027" }, { "id" : 6, - "type" : "stream-exec-vector-search-table-function_1", + "type" : "batch-exec-vector-search-table-function_1", "configuration" : { "table.exec.async-vector-search.max-concurrent-operations" : "10", "table.exec.async-vector-search.output-mode" : "ORDERED", diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/batch-exec-vector-search-table-function_1/sync-vector-search/plan/sync-vector-search.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/batch-exec-vector-search-table-function_1/sync-vector-search/plan/sync-vector-search.json index 3fea189cde038..2382525338c6e 100644 --- a/flink-table/flink-table-planner/src/test/resources/restore-tests/batch-exec-vector-search-table-function_1/sync-vector-search/plan/sync-vector-search.json +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/batch-exec-vector-search-table-function_1/sync-vector-search/plan/sync-vector-search.json @@ -24,10 +24,10 @@ }, "outputType" : "ROW<`id` BIGINT, `content` VARCHAR(2147483647), `vector` ARRAY>", "description" : "TableSourceScan(table=[[default_catalog, default_database, src_t]], fields=[id, content, vector])", - "dynamicFilteringDataListenerID" : "5e850a71-a459-4906-9822-22c324f6f4c9" + "dynamicFilteringDataListenerID" : "6bd08b9c-a3ed-4e52-8349-babb1e553ced" }, { "id" : 2, - "type" : "stream-exec-vector-search-table-function_1", + "type" : "batch-exec-vector-search-table-function_1", "configuration" : { "table.exec.async-vector-search.max-concurrent-operations" : "10", "table.exec.async-vector-search.output-mode" : "ORDERED", diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/batch-exec-vector-search-table-function_1/vector-search-with-runtime-config/plan/vector-search-with-runtime-config.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/batch-exec-vector-search-table-function_1/vector-search-with-runtime-config/plan/vector-search-with-runtime-config.json index 81b762fd38d81..3940bdda18394 100644 --- a/flink-table/flink-table-planner/src/test/resources/restore-tests/batch-exec-vector-search-table-function_1/vector-search-with-runtime-config/plan/vector-search-with-runtime-config.json +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/batch-exec-vector-search-table-function_1/vector-search-with-runtime-config/plan/vector-search-with-runtime-config.json @@ -24,10 +24,10 @@ }, "outputType" : "ROW<`id` BIGINT, `content` VARCHAR(2147483647), `vector` ARRAY>", "description" : "TableSourceScan(table=[[default_catalog, default_database, src_t]], fields=[id, content, vector])", - "dynamicFilteringDataListenerID" : "21fdf54d-fce6-42da-bcd3-a522ebee5cd6" + "dynamicFilteringDataListenerID" : "24c9df65-8b48-4377-a315-b6bbff3e6333" }, { "id" : 10, - "type" : "stream-exec-vector-search-table-function_1", + "type" : "batch-exec-vector-search-table-function_1", "configuration" : { "table.exec.async-vector-search.max-concurrent-operations" : "10", "table.exec.async-vector-search.output-mode" : "ORDERED", From eb65369a2d5af341d7dc4e6227572a92ef777763 Mon Sep 17 00:00:00 2001 From: Wren Chan Date: Tue, 9 Dec 2025 20:47:12 -0500 Subject: [PATCH 24/31] [FLINK-38797][python] Fixed CsvSchemaBuilder.set_null_value to return self This closes #27331. --- .../pyflink/datastream/formats/csv.py | 1 + .../datastream/formats/tests/test_csv.py | 24 +++++++++++++++++++ 2 files changed, 25 insertions(+) diff --git a/flink-python/pyflink/datastream/formats/csv.py b/flink-python/pyflink/datastream/formats/csv.py index 53f85a15441b4..d7da4a03afd76 100644 --- a/flink-python/pyflink/datastream/formats/csv.py +++ b/flink-python/pyflink/datastream/formats/csv.py @@ -237,6 +237,7 @@ def set_null_value(self, null_value: str): Set literal for null value, default to empty sequence. """ self._j_schema_builder.setNullValue(null_value) + return self def disable_quote_char(self): """ diff --git a/flink-python/pyflink/datastream/formats/tests/test_csv.py b/flink-python/pyflink/datastream/formats/tests/test_csv.py index 97d32826020c2..a088cfb502ca9 100644 --- a/flink-python/pyflink/datastream/formats/tests/test_csv.py +++ b/flink-python/pyflink/datastream/formats/tests/test_csv.py @@ -77,6 +77,12 @@ def test_csv_strict_headers(self): self.env.execute('test_csv_strict_headers') _check_csv_strict_headers_results(self, self.test_sink.get_results(True, False)) + def test_csv_default_null_value(self): + schema, lines = _create_csv_default_null_value_schema_and_lines() + self._build_csv_job(schema, lines) + self.env.execute('test_csv_default_null_value') + _check_csv_default_null_value_results(self, self.test_sink.get_results(True, False)) + def test_csv_default_quote_char(self): schema, lines = _create_csv_default_quote_char_schema_and_lines() self._build_csv_job(schema, lines) @@ -344,6 +350,24 @@ def _check_csv_use_header_results(test, results): test.assertEqual(row['number'], 123) +def _create_csv_default_null_value_schema_and_lines() -> Tuple[CsvSchema, List[str]]: + schema = CsvSchema.builder() \ + .add_string_column('string') \ + .add_number_column('number') \ + .set_null_value('') \ + .build() + lines = [ + ',123\n' + ] + return schema, lines + + +def _check_csv_default_null_value_results(test, results): + row = results[0] + test.assertEqual(row['string'], None) + test.assertEqual(row['number'], 123) + + def _create_csv_strict_headers_schema_and_lines() -> Tuple[CsvSchema, List[str]]: schema = CsvSchema.builder() \ .add_string_column('string') \ From 932aa4306efadc8107a33da5efa184a7c71dd14e Mon Sep 17 00:00:00 2001 From: dylanhz <53137516+dylanhz@users.noreply.github.com> Date: Thu, 25 Dec 2025 09:51:30 +0800 Subject: [PATCH 25/31] [FLINK-38824][table] Fix incorrect default values for primitive types This closes #27363. --- .../table/planner/codegen/CodeGenUtils.scala | 4 +++- .../planner/codegen/CodeGenUtilsTest.scala | 8 +++---- .../runtime/stream/sql/CalcITCase.scala | 22 +++++++++++++++++++ 3 files changed, 29 insertions(+), 5 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala index 537adb0746696..c4fac01871059 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala @@ -301,7 +301,9 @@ object CodeGenUtils { // ordered by type root definition case CHAR | VARCHAR => s"$BINARY_STRING.EMPTY_UTF8" case BOOLEAN => "false" - case TINYINT | SMALLINT | INTEGER | DATE | TIME_WITHOUT_TIME_ZONE | INTERVAL_YEAR_MONTH => "-1" + case TINYINT => "((byte) -1)" + case SMALLINT => "((short) -1)" + case INTEGER | DATE | TIME_WITHOUT_TIME_ZONE | INTERVAL_YEAR_MONTH => "-1" case BIGINT | INTERVAL_DAY_TIME => "-1L" case FLOAT => "-1.0f" case DOUBLE => "-1.0d" diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/codegen/CodeGenUtilsTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/codegen/CodeGenUtilsTest.scala index 1df21ef523c44..ae1f6075c6a86 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/codegen/CodeGenUtilsTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/codegen/CodeGenUtilsTest.scala @@ -91,8 +91,8 @@ object CodeGenUtilsTest { java.util.stream.Stream.of( // Basic primitive types Arguments.of(new BooleanType(), "false"), - Arguments.of(new TinyIntType(), "-1"), - Arguments.of(new SmallIntType(), "-1"), + Arguments.of(new TinyIntType(), "((byte) -1)"), + Arguments.of(new SmallIntType(), "((short) -1)"), Arguments.of(new IntType(), "-1"), Arguments.of(new BigIntType(), "-1L"), Arguments.of(new FloatType(), "-1.0f"), @@ -141,12 +141,12 @@ object CodeGenUtilsTest { DistinctType .newBuilder(objectIdentifier, new SmallIntType()) .build(), - "-1"), + "((short) -1)"), Arguments.of( DistinctType .newBuilder(objectIdentifier, new TinyIntType()) .build(), - "-1"), + "((byte) -1)"), Arguments.of( DistinctType .newBuilder(objectIdentifier, new BigIntType()) 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 7d08df3ba556b..f5649f552fadd 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 @@ -926,4 +926,26 @@ class CalcITCase extends StreamingTestBase { assertThat(arr.apply(1).get("nested2").asInstanceOf[Array[String]]) .isEqualTo(Array("Test2", "False")) } + + @Test + def testPrimitiveDefaultValues(): Unit = { + val sql = + """ + |SELECT + | a[1] + | ,b[1] + |FROM ( + | VALUES (CAST(ARRAY[1,2] AS ARRAY), CAST(ARRAY[2,3] AS ARRAY)) + |) t(a, b) + |""".stripMargin + + val result = tEnv + .executeSql(sql) + .collect() + .asScala + .toList + .map(_.toString) + val expected = List("1,2") + assertThat(result).isEqualTo(expected) + } } From 0f27e47dbb2cbbb13e2da54ba7db9c7ea19aac40 Mon Sep 17 00:00:00 2001 From: Zdenek Tison Date: Thu, 20 Nov 2025 15:59:42 +0100 Subject: [PATCH 26/31] [FLINK-38703][runtime] Update slot manager metrics in thread-safety manner --- .../slotmanager/FineGrainedSlotManager.java | 41 ++++++- .../FineGrainedSlotManagerTest.java | 100 ++++++++++++++++++ .../FineGrainedSlotManagerTestBase.java | 6 +- .../metrics/JobManagerMetricsITCase.java | 4 +- 4 files changed, 145 insertions(+), 6 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManager.java index c19943cdaa1cd..f0045fe221c40 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManager.java @@ -56,6 +56,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.StringJoiner; @@ -69,6 +70,8 @@ /** Implementation of {@link SlotManager} supporting fine-grained resource management. */ public class FineGrainedSlotManager implements SlotManager { + public static final Duration METRICS_UPDATE_INTERVAL = Duration.ofSeconds(1); + private static final Logger LOG = LoggerFactory.getLogger(FineGrainedSlotManager.class); private final TaskManagerTracker taskManagerTracker; @@ -114,6 +117,8 @@ public class FineGrainedSlotManager implements SlotManager { @Nullable private ScheduledFuture clusterReconciliationCheck; + @Nullable private ScheduledFuture metricsUpdateFuture; + @Nullable private CompletableFuture requirementsCheckFuture; @Nullable private CompletableFuture declareNeededResourceFuture; @@ -124,6 +129,11 @@ public class FineGrainedSlotManager implements SlotManager { /** True iff the component has been started. */ private boolean started; + /** Metrics. */ + private long lastNumberFreeSlots; + + private long lastNumberRegisteredSlots; + public FineGrainedSlotManager( ScheduledExecutor scheduledExecutor, SlotManagerConfiguration slotManagerConfiguration, @@ -159,6 +169,7 @@ public FineGrainedSlotManager( mainThreadExecutor = null; clusterReconciliationCheck = null; requirementsCheckFuture = null; + metricsUpdateFuture = null; started = false; } @@ -227,10 +238,26 @@ public void start( } private void registerSlotManagerMetrics() { - slotManagerMetricGroup.gauge( - MetricNames.TASK_SLOTS_AVAILABLE, () -> (long) getNumberFreeSlots()); - slotManagerMetricGroup.gauge( - MetricNames.TASK_SLOTS_TOTAL, () -> (long) getNumberRegisteredSlots()); + // Because taskManagerTracker is not thread-safe, metrics must be updated periodically on + // the main thread to prevent concurrent modification issues. + metricsUpdateFuture = + scheduledExecutor.scheduleAtFixedRate( + this::updateMetrics, + 0L, + METRICS_UPDATE_INTERVAL.toMillis(), + TimeUnit.MILLISECONDS); + + slotManagerMetricGroup.gauge(MetricNames.TASK_SLOTS_AVAILABLE, () -> lastNumberFreeSlots); + slotManagerMetricGroup.gauge(MetricNames.TASK_SLOTS_TOTAL, () -> lastNumberRegisteredSlots); + } + + private void updateMetrics() { + Objects.requireNonNull(mainThreadExecutor) + .execute( + () -> { + lastNumberFreeSlots = getNumberFreeSlots(); + lastNumberRegisteredSlots = getNumberRegisteredSlots(); + }); } /** Suspends the component. This clears the internal state of the slot manager. */ @@ -250,6 +277,12 @@ public void suspend() { clusterReconciliationCheck = null; } + // stop the metrics updates + if (metricsUpdateFuture != null) { + metricsUpdateFuture.cancel(false); + metricsUpdateFuture = null; + } + slotStatusSyncer.close(); taskManagerTracker.clear(); resourceTracker.clear(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTest.java index a19f42177b14d..5f374f2f9d162 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTest.java @@ -20,12 +20,14 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple6; +import org.apache.flink.metrics.Gauge; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.clusterframework.types.SlotID; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.metrics.MetricNames; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup; import org.apache.flink.runtime.metrics.util.TestingMetricRegistry; @@ -38,6 +40,7 @@ import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; +import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor; import org.apache.flink.util.function.ThrowingConsumer; import org.junit.jupiter.api.Test; @@ -50,6 +53,7 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import static org.apache.flink.core.testutils.FlinkAssertions.assertThatFuture; @@ -1058,4 +1062,100 @@ void testClearResourceRequirementsWithPendingTaskManager() throws Exception { } }; } + + @Test + void testMetricsUpdate() throws Exception { + final AtomicReference> slotsAvailableGauge = new AtomicReference<>(); + final AtomicReference> slotsTotalGauge = new AtomicReference<>(); + + final MetricRegistry metricRegistry = + TestingMetricRegistry.builder() + .setRegisterConsumer( + (metric, name, group) -> { + if (name.equals(MetricNames.TASK_SLOTS_AVAILABLE)) { + slotsAvailableGauge.set((Gauge) metric); + } else if (name.equals(MetricNames.TASK_SLOTS_TOTAL)) { + slotsTotalGauge.set((Gauge) metric); + } + }) + .build(); + + final Context context = new Context(); + context.setSlotManagerMetricGroup( + SlotManagerMetricGroup.create(metricRegistry, "localhost")); + final ManuallyTriggeredScheduledExecutor scheduledExecutor = + new ManuallyTriggeredScheduledExecutor(); + context.setScheduledExecutor(scheduledExecutor); + final TaskExecutorConnection executorConnection1 = createTaskExecutorConnection(); + final TaskExecutorConnection executorConnection2 = createTaskExecutorConnection(); + + context.runTest( + () -> { + assertThat(slotsAvailableGauge.get().getValue()).isEqualTo(0); + assertThat(slotsTotalGauge.get().getValue()).isEqualTo(0); + + final CompletableFuture + registerTaskManagerFuture1 = new CompletableFuture<>(); + context.runInMainThreadAndWait( + () -> + registerTaskManagerFuture1.complete( + context.getSlotManager() + .registerTaskManager( + executorConnection1, + new SlotReport(), + DEFAULT_TOTAL_RESOURCE_PROFILE, + DEFAULT_SLOT_RESOURCE_PROFILE))); + assertThat(assertFutureCompleteAndReturn(registerTaskManagerFuture1)) + .isEqualTo(SlotManager.RegistrationResult.SUCCESS); + + final CompletableFuture + registerTaskManagerFuture2 = new CompletableFuture<>(); + context.runInMainThreadAndWait( + () -> + registerTaskManagerFuture2.complete( + context.getSlotManager() + .registerTaskManager( + executorConnection2, + new SlotReport( + createAllocatedSlotStatus( + new JobID(), + new AllocationID(), + DEFAULT_SLOT_RESOURCE_PROFILE)), + DEFAULT_TOTAL_RESOURCE_PROFILE, + DEFAULT_SLOT_RESOURCE_PROFILE))); + assertThat(assertFutureCompleteAndReturn(registerTaskManagerFuture2)) + .isEqualTo(SlotManager.RegistrationResult.SUCCESS); + + // triggers the metric update task on the main thread and waits for the main + // thread to process queued up callbacks + scheduledExecutor.triggerPeriodicScheduledTasks(); + context.runInMainThreadAndWait(() -> {}); + + assertThat(slotsTotalGauge.get().getValue()) + .isEqualTo(2 * DEFAULT_NUM_SLOTS_PER_WORKER); + assertThat(slotsAvailableGauge.get().getValue()) + .isEqualTo(2 * DEFAULT_NUM_SLOTS_PER_WORKER - 1); + + final CompletableFuture unRegisterTaskManagerFuture = + new CompletableFuture<>(); + context.runInMainThreadAndWait( + () -> + unRegisterTaskManagerFuture.complete( + context.getSlotManager() + .unregisterTaskManager( + executorConnection2.getInstanceID(), + TEST_EXCEPTION))); + assertThat(assertFutureCompleteAndReturn(unRegisterTaskManagerFuture)).isTrue(); + + // triggers the metric update task on the main thread and waits for the main + // thread to process queued up callbacks + scheduledExecutor.triggerPeriodicScheduledTasks(); + context.runInMainThreadAndWait(() -> {}); + + assertThat(slotsTotalGauge.get().getValue()) + .isEqualTo(DEFAULT_NUM_SLOTS_PER_WORKER); + assertThat(slotsAvailableGauge.get().getValue()) + .isEqualTo(DEFAULT_NUM_SLOTS_PER_WORKER); + }); + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTestBase.java index 977b230863821..0edd01247f766 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTestBase.java @@ -152,7 +152,7 @@ protected class Context { private SlotManagerMetricGroup slotManagerMetricGroup = UnregisteredMetricGroups.createUnregisteredSlotManagerMetricGroup(); private BlockedTaskManagerChecker blockedTaskManagerChecker = resourceID -> false; - private final ScheduledExecutor scheduledExecutor = + private ScheduledExecutor scheduledExecutor = new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()); private final Executor mainThreadExecutor = EXECUTOR_RESOURCE.getExecutor(); private FineGrainedSlotManager slotManager; @@ -193,6 +193,10 @@ public void setBlockedTaskManagerChecker( this.blockedTaskManagerChecker = blockedTaskManagerChecker; } + public void setScheduledExecutor(ScheduledExecutor scheduledExecutor) { + this.scheduledExecutor = scheduledExecutor; + } + void runInMainThread(Runnable runnable) { mainThreadExecutor.execute(runnable); } diff --git a/flink-tests/src/test/java/org/apache/flink/runtime/metrics/JobManagerMetricsITCase.java b/flink-tests/src/test/java/org/apache/flink/runtime/metrics/JobManagerMetricsITCase.java index b231f1a780096..a39ca3c0c0891 100644 --- a/flink-tests/src/test/java/org/apache/flink/runtime/metrics/JobManagerMetricsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/runtime/metrics/JobManagerMetricsITCase.java @@ -31,6 +31,7 @@ import org.apache.flink.metrics.reporter.AbstractReporter; import org.apache.flink.metrics.reporter.MetricReporter; import org.apache.flink.metrics.reporter.MetricReporterFactory; +import org.apache.flink.runtime.resourcemanager.slotmanager.FineGrainedSlotManager; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.PrintSink; @@ -128,7 +129,8 @@ void testJobManagerMetrics() throws Exception { expectedPattern, gaugeNames)); } } - + // wait for metrics update + Thread.sleep(FineGrainedSlotManager.METRICS_UPDATE_INTERVAL.toMillis()); for (Map.Entry, String> entry : reporter.getGauges().entrySet()) { if (entry.getValue().contains(MetricNames.TASK_SLOTS_AVAILABLE)) { assertEquals(0L, entry.getKey().getValue()); From 8647457fb0bb9a7ae3ff6e1c3d53d2b998f343d5 Mon Sep 17 00:00:00 2001 From: Martijn Visser <2989614+MartijnVisser@users.noreply.github.com> Date: Mon, 29 Dec 2025 13:08:42 +0100 Subject: [PATCH 27/31] [FLINK-38914][docs] Preserve page path when linking to stable version The "stable version" link in the warning banner for unreleased/out-of-date docs now redirects to the equivalent page in the stable docs instead of always redirecting to the homepage. --- docs/layouts/partials/docs/inject/content-before.html | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/layouts/partials/docs/inject/content-before.html b/docs/layouts/partials/docs/inject/content-before.html index bfbea6ce7aba4..35d7dde271f05 100644 --- a/docs/layouts/partials/docs/inject/content-before.html +++ b/docs/layouts/partials/docs/inject/content-before.html @@ -20,16 +20,18 @@ Partial that renders before the content on every page. --> {{ if $.Site.Params.ShowOutDatedWarning }} +{{ $stableURL := replaceRE "/flink-docs-[^/]+/" "/flink-docs-stable/" .Permalink }}

- {{ markdownify "This documentation is for an out-of-date version of Apache Flink. We recommend you use the latest [stable version](https://nightlies.apache.org/flink/flink-docs-stable/)."}} + {{ markdownify (printf "This documentation is for an out-of-date version of Apache Flink. We recommend you use the latest [stable version](%s)." $stableURL) }}
{{ end }} {{ if (not $.Site.Params.IsStable) }} +{{ $stableURL := replaceRE "/flink-docs-[^/]+/" "/flink-docs-stable/" .Permalink }}
- {{ markdownify "This documentation is for an unreleased version of Apache Flink. We recommend you use the latest [stable version](https://nightlies.apache.org/flink/flink-docs-stable/)."}} + {{ markdownify (printf "This documentation is for an unreleased version of Apache Flink. We recommend you use the latest [stable version](%s)." $stableURL) }}
{{ end }} From 28bda0b99a414f6cfc2cd9f41ef9b70193fcfb60 Mon Sep 17 00:00:00 2001 From: Martijn Visser <2989614+MartijnVisser@users.noreply.github.com> Date: Wed, 14 Jan 2026 14:42:50 +0100 Subject: [PATCH 28/31] [FLINK-38914][docs] Add canonical tags pointing to stable docs All documentation pages now include a canonical link tag pointing to the stable version (flink-docs-stable). This prevents search engines from indexing duplicate content across master, release, and stable versions, consolidating SEO value on the stable docs where users should land. --- docs/layouts/partials/docs/inject/head.html | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/docs/layouts/partials/docs/inject/head.html b/docs/layouts/partials/docs/inject/head.html index f456c36bdc29f..822b64ae3f392 100644 --- a/docs/layouts/partials/docs/inject/head.html +++ b/docs/layouts/partials/docs/inject/head.html @@ -19,6 +19,16 @@ + +{{- $basePath := (urls.Parse .Site.BaseURL).Path -}} +{{- $pagePath := strings.TrimPrefix $basePath .RelPermalink -}} +{{- if eq $pagePath "" -}} + {{- $pagePath = "/" -}} +{{- else if not (hasPrefix $pagePath "/") -}} + {{- $pagePath = printf "/%s" $pagePath -}} +{{- end -}} + + From 0a836693cdfdad19c7dfae1e72e0545632cb9a03 Mon Sep 17 00:00:00 2001 From: Martijn Visser <2989614+MartijnVisser@users.noreply.github.com> Date: Thu, 15 Jan 2026 15:40:09 +0100 Subject: [PATCH 29/31] [FLINK-38925][docs] Update Matomo URL to the right domain (cherry picked from commit 5da177dfd7c20c540afb9f3987eefd6ad6d554bd) --- .github/workflows/docs.sh | 2 +- docs/layouts/_default/baseof.html | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/docs.sh b/.github/workflows/docs.sh index d5a9bd5a60895..6430f220aa288 100755 --- a/.github/workflows/docs.sh +++ b/.github/workflows/docs.sh @@ -62,7 +62,7 @@ mvn javadoc:aggregate -B \ -Dcheckstyle.skip=true \ -Dspotless.check.skip=true \ -Denforcer.skip=true \ - -Dheader="

Back to Flink Website

" + -Dheader="

Back to Flink Website

" mv target/site/apidocs docs/target/api/java # build python docs diff --git a/docs/layouts/_default/baseof.html b/docs/layouts/_default/baseof.html index f9b76002fa0c1..7ce1dad33e37f 100644 --- a/docs/layouts/_default/baseof.html +++ b/docs/layouts/_default/baseof.html @@ -34,7 +34,7 @@ _paq.push(['trackPageView']); _paq.push(['enableLinkTracking']); (function() { - var u="//matomo.privacy.apache.org/"; + var u="//analytics.apache.org/"; _paq.push(['setTrackerUrl', u+'matomo.php']); _paq.push(['setSiteId', '1']); var d=document, g=d.createElement('script'), s=d.getElementsByTagName('script')[0]; From 00c6e94c1853c6dc11ed8ed94a1ffba6de378ce5 Mon Sep 17 00:00:00 2001 From: Martijn Visser <2989614+MartijnVisser@users.noreply.github.com> Date: Thu, 15 Jan 2026 13:56:27 +0100 Subject: [PATCH 30/31] [FLINK-38924][docs] Redirect users to documentation home page when encountering a 404 (cherry picked from commit 1047406a76620bf8ee5898c795b0009fb8fb5d64) --- docs/layouts/404.html | 119 ++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 119 insertions(+) create mode 100644 docs/layouts/404.html diff --git a/docs/layouts/404.html b/docs/layouts/404.html new file mode 100644 index 0000000000000..0a90ca9a5fb6b --- /dev/null +++ b/docs/layouts/404.html @@ -0,0 +1,119 @@ + + +{{- $stableUrl := "https://nightlies.apache.org/flink/flink-docs-stable/" -}} +{{- $stableUrlZh := "https://nightlies.apache.org/flink/flink-docs-stable/zh/" -}} +{{- $isZh := eq .Site.Language.Lang "zh" -}} + + + + {{ hugo.Generator }} + {{ partial "docs/html-head" . }} + {{ partial "docs/inject/head" . }} + + + + + + +
+ + +
+
+ {{ partial "docs/header" . }} +
+ +
+ {{- if $isZh }} +

页面未找到

+

我们无法在当前版本中找到此页面。这有时发生在页面在版本之间移动或内容是特定版本的情况下。尝试使用搜索来查找您要找的内容。

+
+
+ 查看最新版本文档 + 推荐大多数用户使用。查看稳定版文档。 +
+
+ 留在当前版本 + 继续浏览 {{ .Site.Params.VersionTitle }} 开发文档。 +
+
+ {{- else }} +

Page Not Found

+

We couldn't find this page in the current version. This sometimes happens when pages move between releases or when content is version-specific. Try using the search to find what you're looking for.

+
+
+ View Latest Release Docs + Recommended for most users. View the stable documentation. +
+
+ Stay on This Version + Continue browsing {{ .Site.Params.VersionTitle }} development docs. +
+
+ {{- end }} +
+ + {{ partial "docs/inject/content-after" . }} + +
+ {{ partial "docs/footer" . }} + {{ partial "docs/inject/footer" . }} +
+ + +
+
+ + {{ partial "docs/inject/body" . }} + + + From 0280f6b2013494aa260ce55c641b21958ee234b4 Mon Sep 17 00:00:00 2001 From: leboop <32377468+JavaLeb@users.noreply.github.com> Date: Tue, 20 Jan 2026 09:33:38 +0800 Subject: [PATCH 31/31] Update datagen.md Change the type of the "fields.#.null-rate" option in the DataGen SQL Connector to Float --- docs/content.zh/docs/connectors/table/datagen.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content.zh/docs/connectors/table/datagen.md b/docs/content.zh/docs/connectors/table/datagen.md index 97e7cfee51c10..4fdaf207ea95a 100644 --- a/docs/content.zh/docs/connectors/table/datagen.md +++ b/docs/content.zh/docs/connectors/table/datagen.md @@ -353,7 +353,7 @@ CREATE TABLE Orders (
fields.#.null-rate
optional 0 - (Type of field) + Float 空值比例。